Pārlūkot izejas kodu

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 gadi atpakaļ
vecāks
revīzija
0b4231f5bc
25 mainītis faili ar 1251 papildinājumiem un 212 dzēšanām
  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
     sending it down to the compressor so that each write call doesn't
     compress. (Chris Douglas via acmurthy) 
     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
 Release 0.15.3 - 2008-01-18
 
 
   BUG FIXES
   BUG FIXES

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

@@ -412,13 +412,13 @@ public class DataBlockScanner implements Runnable {
                  StringUtils.stringifyException(e));
                  StringUtils.stringifyException(e));
         
         
         if (second) {
         if (second) {
-          datanode.getMetrics().verificationFailures(1);
+          datanode.getMetrics().blockVerificationFailures.inc(); 
           handleScanFailure(block);
           handleScanFailure(block);
           return;
           return;
         } 
         } 
       } finally {
       } finally {
         IOUtils.closeStream(blockSender);
         IOUtils.closeStream(blockSender);
-        datanode.getMetrics().verifiedBlocks(1);
+        datanode.getMetrics().blocksVerified.inc();
         totalScans++;
         totalScans++;
         totalVerifications++;
         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.io.Text;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 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.BlockCommand;
 import org.apache.hadoop.dfs.DatanodeProtocol;
 import org.apache.hadoop.dfs.DatanodeProtocol;
 import org.apache.hadoop.dfs.FSDatasetInterface.MetaDataInputStream;
 import org.apache.hadoop.dfs.FSDatasetInterface.MetaDataInputStream;
+import org.apache.hadoop.dfs.datanode.metrics.DataNodeMetrics;
 
 
 import java.io.*;
 import java.io.*;
 import java.net.*;
 import java.net.*;
@@ -44,11 +44,6 @@ import java.util.concurrent.Semaphore;
 import java.security.NoSuchAlgorithmException;
 import java.security.NoSuchAlgorithmException;
 import java.security.SecureRandom;
 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
  * DataNode is a class (and program) that stores a set of
  * blocks for a DFS deployment.  A single deployment can
  * blocks for a DFS deployment.  A single deployment can
@@ -150,88 +145,9 @@ public class DataNode implements FSConstants, Runnable {
     return System.currentTimeMillis();
     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.
    * Create the DataNode given a configuration and an array of dataDirs.
@@ -240,7 +156,6 @@ public class DataNode implements FSConstants, Runnable {
   DataNode(Configuration conf, 
   DataNode(Configuration conf, 
            AbstractList<File> dataDirs) throws IOException {
            AbstractList<File> dataDirs) throws IOException {
       
       
-    myMetrics = new DataNodeMetrics(conf);
     datanodeObject = this;
     datanodeObject = this;
 
 
     try {
     try {
@@ -301,7 +216,9 @@ public class DataNode implements FSConstants, Runnable {
         setNewStorageID(dnRegistration);
         setNewStorageID(dnRegistration);
         dnRegistration.storageInfo.layoutVersion = FSConstants.LAYOUT_VERSION;
         dnRegistration.storageInfo.layoutVersion = FSConstants.LAYOUT_VERSION;
         dnRegistration.storageInfo.namespaceID = nsInfo.namespaceID;
         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 {
         try {
           //Equivalent of following (can't do because Simulated is in test dir)
           //Equivalent of following (can't do because Simulated is in test dir)
           //  this.data = new SimulatedFSDataset(conf);
           //  this.data = new SimulatedFSDataset(conf);
@@ -318,6 +235,7 @@ public class DataNode implements FSConstants, Runnable {
       // initialize data node internal structure
       // initialize data node internal structure
       this.data = new FSDataset(storage, conf);
       this.data = new FSDataset(storage, conf);
     }
     }
+
       
       
     // find free port
     // find free port
     ServerSocket ss = new ServerSocket(tmpPort, 0, socAddr.getAddress());
     ServerSocket ss = new ServerSocket(tmpPort, 0, socAddr.getAddress());
@@ -384,6 +302,7 @@ public class DataNode implements FSConstants, Runnable {
       networkLoc = getNetworkLoc(conf);
       networkLoc = getNetworkLoc(conf);
     // register datanode
     // register datanode
     register();
     register();
+    myMetrics = new DataNodeMetrics(conf, dnRegistration.getStorageID());
   }
   }
 
 
   private NamespaceInfo handshake() throws IOException {
   private NamespaceInfo handshake() throws IOException {
@@ -578,6 +497,12 @@ public class DataNode implements FSConstants, Runnable {
       } catch (InterruptedException ie) {
       } 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) {
     while (shouldRun) {
       try {
       try {
-        long now = System.currentTimeMillis();
+        long startTime = now();
 
 
         //
         //
         // Every so often, send heartbeat or block-report
         // Every so often, send heartbeat or block-report
         //
         //
-        if (now - lastHeartbeat > heartBeatInterval) {
+        
+        if (startTime - lastHeartbeat > heartBeatInterval) {
           //
           //
           // All heartbeat messages include following info:
           // All heartbeat messages include following info:
           // -- Datanode name
           // -- Datanode name
@@ -657,8 +583,9 @@ public class DataNode implements FSConstants, Runnable {
                                                        data.getRemaining(),
                                                        data.getRemaining(),
                                                        xmitsInProgress,
                                                        xmitsInProgress,
                                                        xceiverCount.getValue());
                                                        xceiverCount.getValue());
+          myMetrics.heartbeats.inc(now() - startTime);
           //LOG.info("Just sent heartbeat, with name " + localName);
           //LOG.info("Just sent heartbeat, with name " + localName);
-          lastHeartbeat = now;
+          lastHeartbeat = startTime;
           if (!processCommand(cmd))
           if (!processCommand(cmd))
             continue;
             continue;
         }
         }
@@ -697,7 +624,7 @@ public class DataNode implements FSConstants, Runnable {
         }
         }
 
 
         // send block report
         // send block report
-        if (now - lastBlockReport > blockReportInterval) {
+        if (startTime - lastBlockReport > blockReportInterval) {
           //
           //
           // Send latest blockinfo report if timer has expired.
           // Send latest blockinfo report if timer has expired.
           // Get back a list of local block(s) that are obsolete
           // 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,
           DatanodeCommand cmd = namenode.blockReport(dnRegistration,
                   BlockListAsLongs.convertToArrayLongs(bReport));
                   BlockListAsLongs.convertToArrayLongs(bReport));
           long brTime = now() - brStartTime;
           long brTime = now() - brStartTime;
+          myMetrics.blockReports.inc(brTime);
           LOG.info("BlockReport of " + bReport.length +
           LOG.info("BlockReport of " + bReport.length +
               " blocks got processed in " + brTime + " msecs");
               " blocks got processed in " + brTime + " msecs");
           //
           //
@@ -715,10 +643,10 @@ public class DataNode implements FSConstants, Runnable {
           // time before we start the periodic block reports.
           // time before we start the periodic block reports.
           //
           //
           if (resetBlockReportTime) {
           if (resetBlockReportTime) {
-            lastBlockReport = now - new Random().nextInt((int)(blockReportInterval));
+            lastBlockReport = startTime - new Random().nextInt((int)(blockReportInterval));
             resetBlockReportTime = false;
             resetBlockReportTime = false;
           } else {
           } else {
-            lastBlockReport = now;
+            lastBlockReport = startTime;
           }
           }
           processCommand(cmd);
           processCommand(cmd);
         }
         }
@@ -784,7 +712,7 @@ public class DataNode implements FSConstants, Runnable {
         checkDiskError();
         checkDiskError();
         throw e;
         throw e;
       }
       }
-      myMetrics.removedBlocks(toDelete.length);
+      myMetrics.blocksRemoved.inc(toDelete.length);
       break;
       break;
     case DatanodeProtocol.DNA_SHUTDOWN:
     case DatanodeProtocol.DNA_SHUTDOWN:
       // shut down the data node
       // shut down the data node
@@ -981,24 +909,37 @@ public class DataNode implements FSConstants, Runnable {
         if ( version != DATA_TRANFER_VERSION ) {
         if ( version != DATA_TRANFER_VERSION ) {
           throw new IOException( "Version Mismatch" );
           throw new IOException( "Version Mismatch" );
         }
         }
-
+        boolean local = s.getInetAddress().equals(s.getLocalAddress());
         byte op = in.readByte();
         byte op = in.readByte();
-
+        long startTime = now();
         switch ( op ) {
         switch ( op ) {
         case OP_READ_BLOCK:
         case OP_READ_BLOCK:
           readBlock( in );
           readBlock( in );
+          myMetrics.readBlockOp.inc(now() - startTime);
+          if (local)
+            myMetrics.readsFromLocalClient.inc();
+          else
+            myMetrics.readsFromRemoteClient.inc();
           break;
           break;
         case OP_WRITE_BLOCK:
         case OP_WRITE_BLOCK:
           writeBlock( in );
           writeBlock( in );
+          myMetrics.writeBlockOp.inc(now() - startTime);
+          if (local)
+            myMetrics.writesFromLocalClient.inc();
+          else
+            myMetrics.writesFromRemoteClient.inc();
           break;
           break;
         case OP_READ_METADATA:
         case OP_READ_METADATA:
           readMetadata( in );
           readMetadata( in );
+          myMetrics.readMetadataOp.inc(now() - startTime);
           break;
           break;
         case OP_REPLACE_BLOCK: // for balancing purpose; send to a destination
         case OP_REPLACE_BLOCK: // for balancing purpose; send to a destination
           replaceBlock(in);
           replaceBlock(in);
+          myMetrics.replaceBlockOp.inc(now() - startTime);
           break;
           break;
         case OP_COPY_BLOCK: // for balancing purpose; send to a proxy source
         case OP_COPY_BLOCK: // for balancing purpose; send to a proxy source
           copyBlock(in);
           copyBlock(in);
+          myMetrics.copyBlockOp.inc(now() - startTime);
           break;
           break;
         default:
         default:
           throw new IOException("Unknown opcode " + op + " in data stream");
           throw new IOException("Unknown opcode " + op + " in data stream");
@@ -1056,12 +997,12 @@ public class DataNode implements FSConstants, Runnable {
           } catch (IOException ignored) {}
           } 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());
         LOG.info(dnRegistration + " Served block " + block + " to " + s.getInetAddress());
       } catch ( SocketException ignored ) {
       } catch ( SocketException ignored ) {
         // Its ok for remote side to close the connection anytime.
         // Its ok for remote side to close the connection anytime.
-        myMetrics.readBlocks(1);
+        myMetrics.blocksRead.inc();
       } catch ( IOException ioe ) {
       } catch ( IOException ioe ) {
         /* What exactly should we do here?
         /* What exactly should we do here?
          * Earlier version shutdown() datanode if there is disk error.
          * Earlier version shutdown() datanode if there is disk error.
@@ -1301,8 +1242,8 @@ public class DataNode implements FSConstants, Runnable {
         // then send data
         // then send data
         long read = blockSender.sendBlock(targetOut, balancingThrottler);
         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
         // check the response from target
         receiveResponse(targetSock, 1);
         receiveResponse(targetSock, 1);
@@ -1807,7 +1748,7 @@ public class DataNode implements FSConstants, Runnable {
               receiver.close();
               receiver.close();
               block.setNumBytes(receiver.offsetInBlock);
               block.setNumBytes(receiver.offsetInBlock);
               data.finalizeBlock(block);
               data.finalizeBlock(block);
-              myMetrics.wroteBlocks(1);
+              myMetrics.blocksWritten.inc();
               notifyNamenodeReceivedBlock(block, EMPTY_DEL_HINT);
               notifyNamenodeReceivedBlock(block, EMPTY_DEL_HINT);
               LOG.info("Received block " + block + 
               LOG.info("Received block " + block + 
                        " of size " + block.getNumBytes() + 
                        " of size " + block.getNumBytes() + 
@@ -1891,7 +1832,7 @@ public class DataNode implements FSConstants, Runnable {
               receiver.close();
               receiver.close();
               block.setNumBytes(receiver.offsetInBlock);
               block.setNumBytes(receiver.offsetInBlock);
               data.finalizeBlock(block);
               data.finalizeBlock(block);
-              myMetrics.wroteBlocks(1);
+              myMetrics.blocksWritten.inc();
               notifyNamenodeReceivedBlock(block, EMPTY_DEL_HINT);
               notifyNamenodeReceivedBlock(block, EMPTY_DEL_HINT);
               LOG.info("Received block " + block + 
               LOG.info("Received block " + block + 
                        " of size " + block.getNumBytes() + 
                        " of size " + block.getNumBytes() + 
@@ -2129,7 +2070,7 @@ public class DataNode implements FSConstants, Runnable {
           out.write(buf, 0, len);
           out.write(buf, 0, len);
           // Write checksum
           // Write checksum
           checksumOut.write(buf, len, checksumSize);
           checksumOut.write(buf, len, checksumSize);
-          myMetrics.wroteBytes(len);
+          myMetrics.bytesWritten.inc(len);
         }
         }
       } catch (IOException iex) {
       } catch (IOException iex) {
         checkDiskError(iex);
         checkDiskError(iex);
@@ -2295,7 +2236,7 @@ public class DataNode implements FSConstants, Runnable {
           // Finalize the block. Does this fsync()?
           // Finalize the block. Does this fsync()?
           block.setNumBytes(offsetInBlock);
           block.setNumBytes(offsetInBlock);
           data.finalizeBlock(block);
           data.finalizeBlock(block);
-          myMetrics.wroteBlocks(1);
+          myMetrics.blocksWritten.inc();
         }
         }
 
 
       } catch (IOException ioe) {
       } 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.io.*;
 import java.util.*;
 import java.util.*;
 
 
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
+import javax.management.StandardMBean;
+
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.*;
+import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.conf.*;
+import org.apache.hadoop.dfs.datanode.metrics.FSDatasetMBean;
 
 
 /**************************************************
 /**************************************************
  * FSDataset manages a set of data blocks.  Each block
  * FSDataset manages a set of data blocks.  Each block
@@ -521,6 +527,7 @@ class FSDataset implements FSConstants, FSDatasetInterface {
     volumes.getBlockMap(blockMap);
     volumes.getBlockMap(blockMap);
     blockWriteTimeout = Math.max(
     blockWriteTimeout = Math.max(
          conf.getInt("dfs.datanode.block.write.timeout.sec", 3600), 1) * 1000;
          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+"'}";
     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) {
   public long getBlockSize(Block b) {
     return blockMap.get(b).length();
     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;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 
 /**
 /**
@@ -35,7 +36,7 @@ import org.apache.hadoop.util.DiskChecker.DiskErrorException;
  * SimulatedFSDataset (which simulates data).
  * SimulatedFSDataset (which simulates data).
  *
  *
  */
  */
-public interface FSDatasetInterface {
+public interface FSDatasetInterface extends FSDatasetMBean {
   
   
   
   
   /**
   /**
@@ -79,27 +80,7 @@ public interface FSDatasetInterface {
    * @throws IOException
    * @throws IOException
    */
    */
   public boolean metaFileExists(Block b) 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)
    * Returns the specified block's on-disk length (excluding metadata)
@@ -201,6 +182,11 @@ public interface FSDatasetInterface {
      * Stringifies the name of the storage
      * Stringifies the name of the storage
      */
      */
   public String toString();
   public String toString();
+  
+  /**
+   * Shutdown the FSDataset
+   */
+  public void shutdown();
 
 
   /**
   /**
    * Returns the current offset in the data stream.
    * 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();
     long end = FSNamesystem.now();
     numTransactions++;
     numTransactions++;
     totalTimeTransactions += (end-start);
     totalTimeTransactions += (end-start);
-    metrics.incrNumTransactions(1, (int)(end-start));
+    metrics.transactions.inc((end-start));
   }
   }
 
 
   //
   //
@@ -742,7 +742,7 @@ class FSEditLog {
        this.notifyAll();
        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.conf.*;
 import org.apache.hadoop.dfs.BlocksWithLocations.BlockWithLocations;
 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.UnixUserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.mapred.StatusHttpServer;
 import org.apache.hadoop.mapred.StatusHttpServer;
+import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -41,6 +43,9 @@ import java.util.*;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 import java.text.SimpleDateFormat;
 import java.text.SimpleDateFormat;
 
 
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
+import javax.management.StandardMBean;
 import javax.security.auth.login.LoginException;
 import javax.security.auth.login.LoginException;
 
 
 /***************************************************
 /***************************************************
@@ -55,7 +60,7 @@ import javax.security.auth.login.LoginException;
  * 4)  machine --> blocklist (inverted #2)
  * 4)  machine --> blocklist (inverted #2)
  * 5)  LRU cache of updated-heartbeat machines
  * 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");
   public static final Log LOG = LogFactory.getLog("org.apache.hadoop.fs.FSNamesystem");
 
 
   private boolean isPermissionEnabled;
   private boolean isPermissionEnabled;
@@ -259,6 +264,8 @@ class FSNamesystem implements FSConstants {
     hbthread.start();
     hbthread.start();
     lmthread.start();
     lmthread.start();
     replthread.start();
     replthread.start();
+    
+    this.registerMBean(); // register the MBean for the FSNamesystemStutus
 
 
 
 
     this.hostsReader = new HostsFileReader(conf.get("dfs.hosts",""),
     this.hostsReader = new HostsFileReader(conf.get("dfs.hosts",""),
@@ -2743,14 +2750,15 @@ class FSNamesystem implements FSConstants {
   long[] getStats() throws IOException {
   long[] getStats() throws IOException {
     checkSuperuserPrivilege();
     checkSuperuserPrivilege();
     synchronized(heartbeats) {
     synchronized(heartbeats) {
-      return new long[]{totalCapacity(), totalDfsUsed(), totalRemaining()};
+      return new long[]
+            {getCapacityTotal(), getCapacityUsed(), getCapacityRemaining()};
     }
     }
   }
   }
 
 
   /**
   /**
    * Total raw bytes including non-dfs used space.
    * Total raw bytes including non-dfs used space.
    */
    */
-  public long totalCapacity() {
+  public long getCapacityTotal() {
     synchronized (heartbeats) {
     synchronized (heartbeats) {
       return totalCapacity;
       return totalCapacity;
     }
     }
@@ -2759,7 +2767,7 @@ class FSNamesystem implements FSConstants {
   /**
   /**
    * Total used space by data nodes
    * Total used space by data nodes
    */
    */
-  public long totalDfsUsed() {
+  public long getCapacityUsed() {
     synchronized(heartbeats){
     synchronized(heartbeats){
       return totalUsed;
       return totalUsed;
     }
     }
@@ -2767,7 +2775,7 @@ class FSNamesystem implements FSConstants {
   /**
   /**
    * Total non-used raw bytes.
    * Total non-used raw bytes.
    */
    */
-  public long totalRemaining() {
+  public long getCapacityRemaining() {
     synchronized (heartbeats) {
     synchronized (heartbeats) {
       return totalRemaining;
       return totalRemaining;
     }
     }
@@ -3800,7 +3808,7 @@ class FSNamesystem implements FSConstants {
   /**
   /**
    * Get the total number of blocks in the system. 
    * Get the total number of blocks in the system. 
    */
    */
-  long getBlockTotal() {
+  public long getBlocksTotal() {
     return blocksMap.size();
     return blocksMap.size();
   }
   }
 
 
@@ -4001,7 +4009,7 @@ class FSNamesystem implements FSConstants {
    */
    */
   void checkFsObjectLimit() throws IOException {
   void checkFsObjectLimit() throws IOException {
     if (maxFsObjects != 0 &&
     if (maxFsObjects != 0 &&
-        maxFsObjects <= dir.totalInodes() + getBlockTotal()) {
+        maxFsObjects <= dir.totalInodes() + getBlocksTotal()) {
       throw new IOException("Exceeded the configured number of objects " +
       throw new IOException("Exceeded the configured number of objects " +
                              maxFsObjects + " in the filesystem.");
                              maxFsObjects + " in the filesystem.");
     }
     }
@@ -4022,4 +4030,78 @@ class FSNamesystem implements FSConstants {
     this.hardLimit = hardLimit; 
     this.hardLimit = hardLimit; 
     this.lmthread.interrupt();
     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() {
   public String getInodeLimitText() {
     long inodes = fsn.dir.totalInodes();
     long inodes = fsn.dir.totalInodes();
-    long blocks = fsn.getBlockTotal();
+    long blocks = fsn.getBlocksTotal();
     long maxobjects = fsn.getMaxObjects();
     long maxobjects = fsn.getMaxObjects();
     long totalMemory = Runtime.getRuntime().totalMemory();   
     long totalMemory = Runtime.getRuntime().totalMemory();   
     long maxMemory = Runtime.getRuntime().maxMemory();   
     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() {
   public static NameNodeMetrics getNameNodeMetrics() {
     return myMetrics;
     return myMetrics;
   }
   }
+  
     
     
   /**
   /**
    * Initialize the server
    * Initialize the server
@@ -199,6 +200,12 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     if(namesystem != null) namesystem.close();
     if(namesystem != null) namesystem.close();
     if(emptier != null) emptier.interrupt();
     if(emptier != null) emptier.interrupt();
     if(server != null) server.stop();
     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) {
     if (result == null) {
       throw new IOException("Cannot open filename " + src);
       throw new IOException("Cannot open filename " + src);
     }
     }
-    myMetrics.openFile();
+    myMetrics.numFilesOpened.inc();
     return result;
     return result;
   }
   }
 
 
@@ -273,7 +280,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     namesystem.startFile(src,
     namesystem.startFile(src,
         new PermissionStatus(Server.getUserInfo().getUserName(), null, masked),
         new PermissionStatus(Server.getUserInfo().getUserName(), null, masked),
         clientName, clientMachine, overwrite, replication, blockSize);
         clientName, clientMachine, overwrite, replication, blockSize);
-    myMetrics.createFile();
+    myMetrics.numFilesCreated.inc();
   }
   }
 
 
   public boolean setReplication(String src, 
   public boolean setReplication(String src, 
@@ -367,7 +374,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     }
     }
     boolean ret = namesystem.renameTo(src, dst);
     boolean ret = namesystem.renameTo(src, dst);
     if (ret) {
     if (ret) {
-      myMetrics.renameFile();
+      myMetrics.numFilesRenamed.inc();
     }
     }
     return ret;
     return ret;
   }
   }
@@ -425,7 +432,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   public DFSFileInfo[] getListing(String src) throws IOException {
   public DFSFileInfo[] getListing(String src) throws IOException {
     DFSFileInfo[] files = namesystem.getListing(src);
     DFSFileInfo[] files = namesystem.getListing(src);
     if (files != null) {
     if (files != null) {
-      myMetrics.listFile(files.length);
+      myMetrics.numFilesListed.inc(files.length);
     }
     }
     return files;
     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.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 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.*;
 import org.apache.hadoop.metrics.jvm.JvmMetrics;
 import org.apache.hadoop.metrics.jvm.JvmMetrics;
 import org.apache.hadoop.metrics.util.MetricsIntValue;
 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 static Log log = LogFactory.getLog(NameNodeMetrics.class);
     private final MetricsRecord metricsRecord;
     private final MetricsRecord metricsRecord;
     
     
+    private NameNodeStatistics namenodeStats;
+    
     public MetricsTimeVaryingInt numFilesCreated = new MetricsTimeVaryingInt("FilesCreated");
     public MetricsTimeVaryingInt numFilesCreated = new MetricsTimeVaryingInt("FilesCreated");
     public MetricsTimeVaryingInt numFilesOpened = new MetricsTimeVaryingInt("FilesOpened");
     public MetricsTimeVaryingInt numFilesOpened = new MetricsTimeVaryingInt("FilesOpened");
     public MetricsTimeVaryingInt numFilesRenamed = new MetricsTimeVaryingInt("FilesRenamed");
     public MetricsTimeVaryingInt numFilesRenamed = new MetricsTimeVaryingInt("FilesRenamed");
@@ -63,8 +65,8 @@ public class NameNodeMetrics implements Updater {
       JvmMetrics.init("NameNode", sessionId);
       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
       // Create a record for NameNode metrics
       MetricsContext metricsContext = MetricsUtil.getContext("dfs");
       MetricsContext metricsContext = MetricsUtil.getContext("dfs");
@@ -74,6 +76,13 @@ public class NameNodeMetrics implements Updater {
       log.info("Initializing NameNodeMeterics using context object:" +
       log.info("Initializing NameNodeMeterics using context object:" +
                 metricsContext.getClass().getName());
                 metricsContext.getClass().getName());
     }
     }
+    
+
+    
+    public void shutdown() {
+      if (namenodeStats != null) 
+        namenodeStats.shutdown();
+    }
       
       
     /**
     /**
      * Since this object is a registered updater, this method will be called
      * Since this object is a registered updater, this method will be called
@@ -95,28 +104,11 @@ public class NameNodeMetrics implements Updater {
       }
       }
       metricsRecord.update();
       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;
 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.dfs.NameNodeMetrics;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 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
  * This is the implementation of the Name Node JMX MBean
  *
  *
  */
  */
-public class NameNodeMgt implements NameNodeMgtMBean {
+public class NameNodeStatistics implements NameNodeStatisticsMBean {
   private NameNodeMetrics myMetrics;
   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;
     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();
     return myMetrics.syncs.getPreviousIntervalNumOps();
   }
   }
 
 
-  /**
-   * @inheritDoc
-   */
-  public String getNameNodeState() {
-    return myNameNode.isInSafeMode() ? "safeMode" : "Operational";
-  }
  
  
   /**
   /**
    * @inheritDoc
    * @inheritDoc
@@ -146,9 +153,7 @@ public class NameNodeMgt implements NameNodeMgtMBean {
    * @inheritDoc
    * @inheritDoc
    */
    */
   public void resetAllMinMax() {
   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 
  * Many of the statistics are sampled and averaged on an interval 
  * which can be specified in the config file.
  * which can be specified in the config file.
  * <p>
  * <p>
@@ -31,20 +32,18 @@ package org.apache.hadoop.dfs.namenode.metrics;
  * of sampled metrics by  specifying the following two lines
  * of sampled metrics by  specifying the following two lines
  *  in the hadoop-meterics.properties file:
  *  in the hadoop-meterics.properties file:
  *  <pre>
  *  <pre>
- *        rpc.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
- *        rpc.period=10
+ *        dfs.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+ *        dfs.period=10
  *  </pre>
  *  </pre>
  *<p>
  *<p>
  * Note that the metrics are collected regardless of the context used.
  * 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
    * 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.maxIdleTime = conf.getInt("ipc.client.maxidletime", 120000);
     this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
     this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
     this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
     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
     // Start the listener here and let it bind to the port
     listener = new Listener();
     listener = new Listener();
     this.port = listener.getAddress().getPort();    
     this.port = listener.getAddress().getPort();    
+    this.rpcMetrics = new RpcMetrics(serverName,
+                          Integer.toString(this.port), this);
+
 
 
     // Create the responder here
     // Create the responder here
     responder = new Responder();
     responder = new Responder();
@@ -1009,6 +1011,9 @@ public abstract class Server {
     listener.doStop();
     listener.doStop();
     responder.interrupt();
     responder.interrupt();
     notifyAll();
     notifyAll();
+    if (this.rpcMetrics != null) {
+      this.rpcMetrics.shutdown();
+    }
   }
   }
 
 
   /** Wait for the server to be stopped.
   /** 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 {
 public class RpcMetrics implements Updater {
   private MetricsRecord metricsRecord;
   private MetricsRecord metricsRecord;
   private static Log LOG = LogFactory.getLog(JvmMetrics.class);
   private static Log LOG = LogFactory.getLog(JvmMetrics.class);
+  RpcMgt rpcMgt;
   
   
   private void setTags(String serverName, String port) {
   private void setTags(String serverName, String port) {
     metricsRecord.setTag("serverName", serverName);
     metricsRecord.setTag("serverName", serverName);
@@ -57,7 +58,7 @@ public class RpcMetrics implements Updater {
     context.registerUpdater(this);
     context.registerUpdater(this);
     
     
     // Need to clean up the interface to RpcMgt - don't need both metrics and server params
     // 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);
     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;
 package org.apache.hadoop.ipc.metrics;
 
 
 
 
+import javax.management.ObjectName;
+
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 
 
@@ -29,14 +31,21 @@ import org.apache.hadoop.metrics.util.MBeanUtil;
 class RpcMgt implements RpcMgtMBean {
 class RpcMgt implements RpcMgtMBean {
   private RpcMetrics myMetrics;
   private RpcMetrics myMetrics;
   private Server myServer;
   private Server myServer;
+  private ObjectName mbeanName;
   
   
   RpcMgt(final String serviceName, final String port,
   RpcMgt(final String serviceName, final String port,
                 final RpcMetrics metrics, Server server) {
                 final RpcMetrics metrics, Server server) {
     myMetrics = metrics;
     myMetrics = metrics;
     myServer = server;
     myServer = server;
-    MBeanUtil.registerMBean(serviceName, "RpcStatistics", this);
+    mbeanName = MBeanUtil.registerMBean(serviceName,
+                    "RpcStatisticsForPort" + port, this);
   }
   }
 
 
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+  
   /**
   /**
    * @inheritDoc
    * @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 java.lang.management.ManagementFactory;
 
 
+import javax.management.InstanceNotFoundException;
 import javax.management.MBeanServer;
 import javax.management.MBeanServer;
 import javax.management.MalformedObjectNameException;
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 import javax.management.ObjectName;
@@ -40,17 +41,33 @@ public class MBeanUtil {
    * @param serviceName
    * @param serviceName
    * @param nameName
    * @param nameName
    * @param theMbean - the MBean to register
    * @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 String nameName,
 		  							final Object theMbean) {
 		  							final Object theMbean) {
     final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
     final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
     ObjectName name = getMBeanName(serviceName, nameName);
     ObjectName name = getMBeanName(serviceName, nameName);
     try {
     try {
       mbs.registerMBean(theMbean, name);
       mbs.registerMBean(theMbean, name);
+      return name;
     } catch (Exception e) {
     } catch (Exception e) {
       e.printStackTrace();
       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,
   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() {
   public synchronized void inc() {
     currentValue++;
     currentValue++;
@@ -76,7 +76,7 @@ public class MetricsTimeVaryingInt {
    */
    */
   public synchronized void pushMetric(final MetricsRecord mr) {
   public synchronized void pushMetric(final MetricsRecord mr) {
     intervalHeartBeat();
     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.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStream;
 import java.util.HashMap;
 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.Configurable;
 import org.apache.hadoop.conf.Configuration;
 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;
 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 HashMap<Block, BInfo> blockMap = null;
   private SimulatedStorage storage = null;
   private SimulatedStorage storage = null;
+  private String storageId;
   
   
   public SimulatedFSDataset(Configuration conf) throws IOException {
   public SimulatedFSDataset(Configuration conf) throws IOException {
     setConf(conf);
     setConf(conf);
@@ -203,12 +212,16 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
   
   
   private SimulatedFSDataset() { // real construction when setConf called.. Uggg
   private SimulatedFSDataset() { // real construction when setConf called.. Uggg
   }
   }
+  
   public Configuration getConf() {
   public Configuration getConf() {
     return conf;
     return conf;
   }
   }
 
 
   public void setConf(Configuration iconf)  {
   public void setConf(Configuration iconf)  {
     conf = iconf;
     conf = iconf;
+    storageId = conf.get("StorageId", "unknownStorageId" +
+                                        new Random().nextInt());
+    registerMBean(storageId);
     storage = new SimulatedStorage(
     storage = new SimulatedStorage(
         conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY));
         conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY));
     //DataNode.LOG.info("Starting Simulated storage; Capacity = " + getCapacity() + 
     //DataNode.LOG.info("Starting Simulated storage; Capacity = " + getCapacity() + 
@@ -312,7 +325,7 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
   }
   }
 
 
   public String toString() {
   public String toString() {
-    return "Simulated FSDataset";
+    return getStorageInfo();
   }
   }
 
 
   public synchronized BlockWriteStreams writeToBlock(Block b, 
   public synchronized BlockWriteStreams writeToBlock(Block b, 
@@ -540,6 +553,38 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
       length += len;
       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
     // wait for number of blocks to decrease
     while (true) {
     while (true) {
-      long total = namesys.getBlockTotal() + namesys.dir.totalInodes();
+      long total = namesys.getBlocksTotal() + namesys.dir.totalInodes();
       System.out.println("Comparing current nodes " + total +
       System.out.println("Comparing current nodes " + total +
                          " to become " + num);
                          " to become " + num);
       if (total == num) {
       if (total == num) {

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

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