You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by dh...@apache.org on 2008/11/11 19:49:30 UTC

svn commit: r713106 - in /hadoop/core/trunk: ./ src/core/org/apache/hadoop/fs/ src/core/org/apache/hadoop/util/ src/hdfs/org/apache/hadoop/hdfs/protocol/ src/hdfs/org/apache/hadoop/hdfs/server/balancer/ src/hdfs/org/apache/hadoop/hdfs/server/namenode/ ...

Author: dhruba
Date: Tue Nov 11 10:49:29 2008
New Revision: 713106

URL: http://svn.apache.org/viewvc?rev=713106&view=rev
Log:
HADOOP-4435. The JobTracker WebUI displays the amount of heap memory 
in use. (dhruba)


Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/core/org/apache/hadoop/fs/FsShell.java
    hadoop/core/trunk/src/core/org/apache/hadoop/util/StringUtils.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/tools/DFSAdmin.java
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ClusterStatus.java
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java
    hadoop/core/trunk/src/webapps/datanode/browseDirectory.jsp
    hadoop/core/trunk/src/webapps/hdfs/dfshealth.jsp
    hadoop/core/trunk/src/webapps/job/jobtracker.jsp

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Tue Nov 11 10:49:29 2008
@@ -25,6 +25,9 @@
     HADOOP-4567. GetFileBlockLocations returns the NetworkTopology
     information of the machines where the blocks reside. (dhruba)
 
+    HADOOP-4435. The JobTracker WebUI displays the amount of heap memory 
+    in use. (dhruba)
+
   NEW FEATURES
 
     HADOOP-4575. Add a proxy service for relaying HsftpFileSystem requests.

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/fs/FsShell.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/fs/FsShell.java?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/fs/FsShell.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/fs/FsShell.java Tue Nov 11 10:49:29 2008
@@ -43,6 +43,7 @@
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 
 /** Provide command line access to a FileSystem. */
@@ -63,12 +64,6 @@
   static final String COPYTOLOCAL_SHORT_USAGE = GET_SHORT_USAGE.replace(
       "-get", "-copyToLocal");
   static final String TAIL_USAGE="-tail [-f] <file>";
-  private static final DecimalFormat decimalFormat;
-  static {
-	  NumberFormat numberFormat = NumberFormat.getNumberInstance(Locale.ENGLISH);
-	  decimalFormat = (DecimalFormat) numberFormat;
-	  decimalFormat.applyPattern("#.##");
-  }
 
   /**
    */
@@ -1216,31 +1211,19 @@
   
   /**
    * Return an abbreviated English-language desc of the byte length
+   * @deprecated Consider using {@link org.apache.hadoop.util.StringUtils#byteDesc} instead.
    */
+  @Deprecated
   public static String byteDesc(long len) {
-    double val = 0.0;
-    String ending = "";
-    if (len < 1024 * 1024) {
-      val = (1.0 * len) / 1024;
-      ending = " KB";
-    } else if (len < 1024 * 1024 * 1024) {
-      val = (1.0 * len) / (1024 * 1024);
-      ending = " MB";
-    } else if (len < 1024L * 1024 * 1024 * 1024) {
-      val = (1.0 * len) / (1024 * 1024 * 1024);
-      ending = " GB";
-    } else if (len < 1024L * 1024 * 1024 * 1024 * 1024) {
-      val = (1.0 * len) / (1024L * 1024 * 1024 * 1024);
-      ending = " TB";
-    } else {
-      val = (1.0 * len) / (1024L * 1024 * 1024 * 1024 * 1024);
-      ending = " PB";
-    }
-    return limitDecimalTo2(val) + ending;
+    return StringUtils.byteDesc(len);
   }
 
+  /**
+   * @deprecated Consider using {@link org.apache.hadoop.util.StringUtils#limitDecimalTo2} instead.
+   */
+  @Deprecated
   public static synchronized String limitDecimalTo2(double d) {
-    return decimalFormat.format(d);
+    return StringUtils.limitDecimalTo2(d);
   }
 
   private void printHelp(String cmd) {

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/util/StringUtils.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/util/StringUtils.java?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/util/StringUtils.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/util/StringUtils.java Tue Nov 11 10:49:29 2008
@@ -26,6 +26,8 @@
 import java.net.UnknownHostException;
 import java.text.DateFormat;
 import java.text.DecimalFormat;
+import java.text.NumberFormat;
+import java.util.Locale;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Date;
@@ -40,6 +42,13 @@
  */
 public class StringUtils {
 
+  private static final DecimalFormat decimalFormat;
+  static {
+          NumberFormat numberFormat = NumberFormat.getNumberInstance(Locale.ENGLISH);
+          decimalFormat = (DecimalFormat) numberFormat;
+          decimalFormat.applyPattern("#.##");
+  }
+
   /**
    * Make a string representation of the exception.
    * @param e The exception to stringify
@@ -647,4 +656,33 @@
       
       return sb.toString();
     }
+
+  /**
+   * Return an abbreviated English-language desc of the byte length
+   */
+  public static String byteDesc(long len) {
+    double val = 0.0;
+    String ending = "";
+    if (len < 1024 * 1024) {
+      val = (1.0 * len) / 1024;
+      ending = " KB";
+    } else if (len < 1024 * 1024 * 1024) {
+      val = (1.0 * len) / (1024 * 1024);
+      ending = " MB";
+    } else if (len < 1024L * 1024 * 1024 * 1024) {
+      val = (1.0 * len) / (1024 * 1024 * 1024);
+      ending = " GB";
+    } else if (len < 1024L * 1024 * 1024 * 1024 * 1024) {
+      val = (1.0 * len) / (1024L * 1024 * 1024 * 1024);
+      ending = " TB";
+    } else {
+      val = (1.0 * len) / (1024L * 1024 * 1024 * 1024 * 1024);
+      ending = " PB";
+    }
+    return limitDecimalTo2(val) + ending;
+  }
+
+  public static synchronized String limitDecimalTo2(double d) {
+    return decimalFormat.format(d);
+  }
 }

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java Tue Nov 11 10:49:29 2008
@@ -22,7 +22,6 @@
 import java.io.IOException;
 import java.util.Date;
 
-import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
@@ -31,6 +30,7 @@
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.util.StringUtils;
 
 /** 
  * DatanodeInfo represents the status of a DataNode.
@@ -185,12 +185,12 @@
     } else {
       buffer.append("Normal\n");
     }
-    buffer.append("Configured Capacity: "+c+" ("+FsShell.byteDesc(c)+")"+"\n");
-    buffer.append("DFS Used: "+u+" ("+FsShell.byteDesc(u)+")"+"\n");
-    buffer.append("Non DFS Used: "+nonDFSUsed+" ("+FsShell.byteDesc(nonDFSUsed)+")"+"\n");
-    buffer.append("DFS Remaining: " +r+ "("+FsShell.byteDesc(r)+")"+"\n");
-    buffer.append("DFS Used%: "+FsShell.limitDecimalTo2(usedPercent)+"%\n");
-    buffer.append("DFS Remaining%: "+FsShell.limitDecimalTo2(remainingPercent)+"%\n");
+    buffer.append("Configured Capacity: "+c+" ("+StringUtils.byteDesc(c)+")"+"\n");
+    buffer.append("DFS Used: "+u+" ("+StringUtils.byteDesc(u)+")"+"\n");
+    buffer.append("Non DFS Used: "+nonDFSUsed+" ("+StringUtils.byteDesc(nonDFSUsed)+")"+"\n");
+    buffer.append("DFS Remaining: " +r+ "("+StringUtils.byteDesc(r)+")"+"\n");
+    buffer.append("DFS Used%: "+StringUtils.limitDecimalTo2(usedPercent)+"%\n");
+    buffer.append("DFS Remaining%: "+StringUtils.limitDecimalTo2(remainingPercent)+"%\n");
     buffer.append("Last contact: "+new Date(lastUpdate)+"\n");
     return buffer.toString();
   }
@@ -212,10 +212,10 @@
     } else {
       buffer.append(" IN");
     }
-    buffer.append(" " + c + "(" + FsShell.byteDesc(c)+")");
-    buffer.append(" " + u + "(" + FsShell.byteDesc(u)+")");
-    buffer.append(" " + FsShell.limitDecimalTo2(((1.0*u)/c)*100)+"%");
-    buffer.append(" " + r + "(" + FsShell.byteDesc(r)+")");
+    buffer.append(" " + c + "(" + StringUtils.byteDesc(c)+")");
+    buffer.append(" " + u + "(" + StringUtils.byteDesc(u)+")");
+    buffer.append(" " + StringUtils.limitDecimalTo2(((1.0*u)/c)*100)+"%");
+    buffer.append(" " + r + "(" + StringUtils.byteDesc(r)+")");
     buffer.append(" " + new Date(lastUpdate));
     return buffer.toString();
   }

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java Tue Nov 11 10:49:29 2008
@@ -59,7 +59,6 @@
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
@@ -268,7 +267,7 @@
               addToMoved(block);
               if (LOG.isDebugEnabled()) {
                 LOG.debug("Decided to move block "+ block.getBlockId()
-                    +" with a length of "+FsShell.byteDesc(block.getNumBytes())
+                    +" with a length of "+StringUtils.byteDesc(block.getNumBytes())
                     + " bytes from " + source.getName() 
                     + " to " + target.getName()
                     + " using proxy source " + proxySource.getName() );
@@ -1119,7 +1118,7 @@
       if (!target.isMoveQuotaFull()) {
         targetCandidates.remove();
       }
-      LOG.info("Decided to move "+FsShell.byteDesc(size)+" bytes from "
+      LOG.info("Decided to move "+StringUtils.byteDesc(size)+" bytes from "
           +source.datanode.getName() + " to " + target.datanode.getName());
       return true;
     }
@@ -1167,7 +1166,7 @@
       if ( !source.isMoveQuotaFull()) {
         sourceCandidates.remove();
       }
-      LOG.info("Decided to move "+FsShell.byteDesc(size)+" bytes from "
+      LOG.info("Decided to move "+StringUtils.byteDesc(size)+" bytes from "
           +source.datanode.getName() + " to " + target.datanode.getName());
       return true;
     }
@@ -1396,7 +1395,7 @@
           System.out.println("The cluster is balanced. Exiting...");
           return SUCCESS;
         } else {
-          LOG.info( "Need to move "+ FsShell.byteDesc(bytesLeftToMove)
+          LOG.info( "Need to move "+ StringUtils.byteDesc(bytesLeftToMove)
               +" bytes to make the cluster balanced." );
         }
         
@@ -1410,16 +1409,16 @@
           System.out.println("No block can be moved. Exiting...");
           return NO_MOVE_BLOCK;
         } else {
-          LOG.info( "Will move " + FsShell.byteDesc(bytesToMove) +
+          LOG.info( "Will move " + StringUtils.byteDesc(bytesToMove) +
               "bytes in this iteration");
         }
    
         formatter.format("%-24s %10d  %19s  %18s  %17s\n", 
             DateFormat.getDateTimeInstance().format(new Date()),
             iterations,
-            FsShell.byteDesc(bytesMoved.get()),
-            FsShell.byteDesc(bytesLeftToMove),
-            FsShell.byteDesc(bytesToMove)
+            StringUtils.byteDesc(bytesMoved.get()),
+            StringUtils.byteDesc(bytesLeftToMove),
+            StringUtils.byteDesc(bytesToMove)
             );
         
         /* For each pair of <source, target>, start a thread that repeatedly 

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java Tue Nov 11 10:49:29 2008
@@ -41,7 +41,7 @@
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.*;
 
@@ -211,8 +211,8 @@
       long pct = ((inodes + blocks) * 100)/maxobjects;
       str += " / " + maxobjects + " (" + pct + "%)";
     }
-    str += ".  Heap Size is " + FsShell.byteDesc(totalMemory) + " / " + 
-           FsShell.byteDesc(maxMemory) + 
+    str += ".  Heap Size is " + StringUtils.byteDesc(totalMemory) + " / " + 
+           StringUtils.byteDesc(maxMemory) + 
            " (" + used + "%) <br>";
     return str;
   }

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/tools/DFSAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/tools/DFSAdmin.java?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/tools/DFSAdmin.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/tools/DFSAdmin.java Tue Nov 11 10:49:29 2008
@@ -271,15 +271,15 @@
         System.out.println(status.getStatusText(false));
       }
       System.out.println("Configured Capacity: " + capacity
-                         + " (" + byteDesc(capacity) + ")");
+                         + " (" + StringUtils.byteDesc(capacity) + ")");
       System.out.println("Present Capacity: " + presentCapacity
-          + " (" + byteDesc(presentCapacity) + ")");
+          + " (" + StringUtils.byteDesc(presentCapacity) + ")");
       System.out.println("DFS Remaining: " + remaining
-          + " (" + byteDesc(remaining) + ")");
+          + " (" + StringUtils.byteDesc(remaining) + ")");
       System.out.println("DFS Used: " + used
-                         + " (" + byteDesc(used) + ")");
+                         + " (" + StringUtils.byteDesc(used) + ")");
       System.out.println("DFS Used%: "
-                         + limitDecimalTo2(((1.0 * used) / presentCapacity) * 100)
+                         + StringUtils.limitDecimalTo2(((1.0 * used) / presentCapacity) * 100)
                          + "%");
       System.out.println();
 

Modified: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ClusterStatus.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ClusterStatus.java?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ClusterStatus.java (original)
+++ hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ClusterStatus.java Tue Nov 11 10:49:29 2008
@@ -57,6 +57,8 @@
   private int max_map_tasks;
   private int max_reduce_tasks;
   private JobTracker.State state;
+  private long used_memory;
+  private long max_memory;
 
   ClusterStatus() {}
   
@@ -77,6 +79,8 @@
     max_map_tasks = maxMaps;
     max_reduce_tasks = maxReduces;
     this.state = state;
+    used_memory = Runtime.getRuntime().totalMemory();
+    max_memory = Runtime.getRuntime().maxMemory();
   }
   
 
@@ -135,12 +139,32 @@
     return state;
   }
 
+  /**
+   * Get the total heap memory used by the <code>JobTracker</code>
+   * 
+   * @return the size of heap memory used by the <code>JobTracker</code>
+   */
+  public long getUsedMemory() {
+    return used_memory;
+  }
+
+  /**
+   * Get the maximum configured heap memory that can be used by the <code>JobTracker</code>
+   * 
+   * @return the configured size of max heap memory that can be used by the <code>JobTracker</code>
+   */
+  public long getMaxMemory() {
+    return max_memory;
+  }
+
   public void write(DataOutput out) throws IOException {
     out.writeInt(task_trackers);
     out.writeInt(map_tasks);
     out.writeInt(reduce_tasks);
     out.writeInt(max_map_tasks);
     out.writeInt(max_reduce_tasks);
+    out.writeLong(used_memory);
+    out.writeLong(max_memory);
     WritableUtils.writeEnum(out, state);
   }
 
@@ -150,7 +174,8 @@
     reduce_tasks = in.readInt();
     max_map_tasks = in.readInt();
     max_reduce_tasks = in.readInt();
+    used_memory = in.readLong();
+    max_memory = in.readLong();
     state = WritableUtils.readEnum(in, JobTracker.State.class);
   }
-
 }

Modified: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java (original)
+++ hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java Tue Nov 11 10:49:29 2008
@@ -49,8 +49,10 @@
    * Version 15: Added KILLED status to JobStatus as part of HADOOP-3924            
    * Version 16: Added getSetupTaskReports and 
    *             setupProgress to JobStatus as part of HADOOP-4261           
+   * Version 17: getClusterStatus returns the amount of memory used by 
+   *             the server. HADOOP-4435
    */
-  public static final long versionID = 16L;
+  public static final long versionID = 17L;
 
   /**
    * Allocate a name for the job.

Modified: hadoop/core/trunk/src/webapps/datanode/browseDirectory.jsp
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/webapps/datanode/browseDirectory.jsp?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/webapps/datanode/browseDirectory.jsp (original)
+++ hadoop/core/trunk/src/webapps/datanode/browseDirectory.jsp Tue Nov 11 10:49:29 2008
@@ -107,9 +107,9 @@
           if (files[i].getPath().toString().endsWith(".crc")) continue;
           if (!files[i].isDir()) {
             cols[1] = "file";
-            cols[2] = FsShell.byteDesc(files[i].getLen());
+            cols[2] = StringUtils.byteDesc(files[i].getLen());
             cols[3] = Short.toString(files[i].getReplication());
-            cols[4] = FsShell.byteDesc(files[i].getBlockSize());
+            cols[4] = StringUtils.byteDesc(files[i].getBlockSize());
           }
           else {
             cols[1] = "dir";

Modified: hadoop/core/trunk/src/webapps/hdfs/dfshealth.jsp
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/webapps/hdfs/dfshealth.jsp?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/webapps/hdfs/dfshealth.jsp (original)
+++ hadoop/core/trunk/src/webapps/hdfs/dfshealth.jsp Tue Nov 11 10:49:29 2008
@@ -88,8 +88,8 @@
     long u = d.getDfsUsed();
     long nu = d.getNonDfsUsed();
     long r = d.getRemaining();
-    String percentUsed = FsShell.limitDecimalTo2(d.getDfsUsedPercent());    
-    String percentRemaining = FsShell.limitDecimalTo2(d.getRemainingPercent());    
+    String percentUsed = StringUtils.limitDecimalTo2(d.getDfsUsedPercent());    
+    String percentRemaining = StringUtils.limitDecimalTo2(d.getRemainingPercent());    
     
     String adminState = (d.isDecommissioned() ? "Decommissioned" :
                          (d.isDecommissionInProgress() ? "Decommission In Progress":
@@ -102,13 +102,13 @@
               "<td class=\"adminstate\">" +
               adminState +
               "<td align=\"right\" class=\"capacity\">" +
-              FsShell.limitDecimalTo2(c*1.0/diskBytes) +
+              StringUtils.limitDecimalTo2(c*1.0/diskBytes) +
               "<td align=\"right\" class=\"used\">" +
-              FsShell.limitDecimalTo2(u*1.0/diskBytes) +      
+              StringUtils.limitDecimalTo2(u*1.0/diskBytes) +      
               "<td align=\"right\" class=\"nondfsused\">" +
-              FsShell.limitDecimalTo2(nu*1.0/diskBytes) +      
+              StringUtils.limitDecimalTo2(nu*1.0/diskBytes) +      
               "<td align=\"right\" class=\"remaining\">" +
-              FsShell.limitDecimalTo2(r*1.0/diskBytes) +      
+              StringUtils.limitDecimalTo2(r*1.0/diskBytes) +      
               "<td align=\"right\" class=\"pcused\">" + percentUsed +
               "<td class=\"pcused\">" +
               ServletUtil.percentageGraph( (int)Double.parseDouble(percentUsed) , 100) +
@@ -161,17 +161,17 @@
 
     out.print( "<div id=\"dfstable\"> <table>\n" +
 	       rowTxt() + colTxt() + "Configured Capacity" + colTxt() + ":" + colTxt() +
-	       FsShell.byteDesc( total ) +
+	       StringUtils.byteDesc( total ) +
 	       rowTxt() + colTxt() + "DFS Used" + colTxt() + ":" + colTxt() +
-	       FsShell.byteDesc( used ) +
+	       StringUtils.byteDesc( used ) +
 	       rowTxt() + colTxt() + "Non DFS Used" + colTxt() + ":" + colTxt() +
-	       FsShell.byteDesc( nonDFS ) +
+	       StringUtils.byteDesc( nonDFS ) +
 	       rowTxt() + colTxt() + "DFS Remaining" + colTxt() + ":" + colTxt() +
-	       FsShell.byteDesc( remaining ) +
+	       StringUtils.byteDesc( remaining ) +
 	       rowTxt() + colTxt() + "DFS Used%" + colTxt() + ":" + colTxt() +
-	       FsShell.limitDecimalTo2(percentUsed) + " %" +
+	       StringUtils.limitDecimalTo2(percentUsed) + " %" +
 	       rowTxt() + colTxt() + "DFS Remaining%" + colTxt() + ":" + colTxt() +
-	       FsShell.limitDecimalTo2(percentRemaining) + " %" +
+	       StringUtils.limitDecimalTo2(percentRemaining) + " %" +
 	       rowTxt() + colTxt() +
                "<a href=\"#LiveNodes\">Live Nodes</a> " +
                colTxt() + ":" + colTxt() + live.size() +

Modified: hadoop/core/trunk/src/webapps/job/jobtracker.jsp
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/webapps/job/jobtracker.jsp?rev=713106&r1=713105&r2=713106&view=diff
==============================================================================
--- hadoop/core/trunk/src/webapps/job/jobtracker.jsp (original)
+++ hadoop/core/trunk/src/webapps/job/jobtracker.jsp Tue Nov 11 10:49:29 2008
@@ -10,6 +10,7 @@
 %>
 <%
   JobTracker tracker = (JobTracker) application.getAttribute("job.tracker");
+  ClusterStatus status = tracker.getClusterStatus();
   String trackerName = 
            StringUtils.simpleHostname(tracker.getJobTrackerMachine());
   JobQueueInfo[] queues = tracker.getQueues();
@@ -20,9 +21,8 @@
 <%!
   private static DecimalFormat percentFormat = new DecimalFormat("##0.00");
   
-  public void generateSummaryTable(JspWriter out,
+  public void generateSummaryTable(JspWriter out, ClusterStatus status,
                                    JobTracker tracker) throws IOException {
-    ClusterStatus status = tracker.getClusterStatus();
     String tasksPerNode = status.getTaskTrackers() > 0 ?
       percentFormat.format(((double)(status.getMaxMapTasks() +
                       status.getMaxReduceTasks())) / status.getTaskTrackers()):
@@ -79,7 +79,7 @@
   </ul>
 </div>
 
-<b>State:</b> <%= tracker.getClusterStatus().getJobTrackerState() %><br>
+<b>State:</b> <%= status.getJobTrackerState() %><br>
 <b>Started:</b> <%= new Date(tracker.getStartTime())%><br>
 <b>Version:</b> <%= VersionInfo.getVersion()%>,
                 r<%= VersionInfo.getRevision()%><br>
@@ -88,9 +88,9 @@
 <b>Identifier:</b> <%= tracker.getTrackerIdentifier()%><br>                 
                    
 <hr>
-<h2>Cluster Summary</h2>
+<h2>Cluster Summary (Heap Size is <%= StringUtils.byteDesc(status.getUsedMemory()) %>/<%= StringUtils.byteDesc(status.getMaxMemory()) %>)</h2>
 <% 
- generateSummaryTable(out, tracker); 
+ generateSummaryTable(out, status, tracker); 
 %>
 <hr>
 <h2 id="scheduling_info">Scheduling Information</h2>