Quellcode durchsuchen

Merge trunk into HA branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1235380 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers vor 13 Jahren
Ursprung
Commit
83bcb1bbf4
15 geänderte Dateien mit 110 neuen und 141 gelöschten Zeilen
  1. 0 5
      hadoop-common-project/hadoop-common/src/main/docs/src/documentation/content/xdocs/commands_manual.xml
  2. 4 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  3. 9 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/docs/src/documentation/content/xdocs/hdfs_user_guide.xml
  4. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  5. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BenchmarkThroughput.java
  6. 0 26
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  7. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
  8. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.java
  9. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
  10. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java
  11. 2 0
      hadoop-mapreduce-project/CHANGES.txt
  12. 14 9
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  13. 15 8
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
  14. 52 62
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
  15. 10 10
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm

+ 0 - 5
hadoop-common-project/hadoop-common/src/main/docs/src/documentation/content/xdocs/commands_manual.xml

@@ -753,11 +753,6 @@
 			
 			<section>
 				<title> secondarynamenode </title>
-				<note>
-					The Secondary NameNode has been deprecated. Instead, consider using the
-					<a href="http://hadoop.apache.org/hdfs/docs/current/hdfs_user_guide.html#Checkpoint+Node">Checkpoint Node</a> or 
-					<a href="http://hadoop.apache.org/hdfs/docs/current/hdfs_user_guide.html#Backup+Node">Backup Node</a>. 
-				</note>
 				<p>	
 					Runs the HDFS secondary 
 					namenode. See <a href="http://hadoop.apache.org/hdfs/docs/current/hdfs_user_guide.html#Secondary+NameNode">Secondary NameNode</a> 

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -276,6 +276,8 @@ Release 0.23.1 - UNRELEASED
 
     HDFS-2818. Fix a missing space issue in HDFS webapps' title tags. (Devaraj K via harsh)
 
+    HDFS-2397. Undeprecate SecondaryNameNode (eli)
+
   OPTIMIZATIONS
 
     HDFS-2130. Switch default checksum to CRC32C. (todd)
@@ -359,6 +361,8 @@ Release 0.23.1 - UNRELEASED
     HDFS-2822. processMisReplicatedBlock incorrectly identifies
     under-construction blocks as under-replicated. (todd)
 
+    HDFS-442. dfsthroughput in test jar throws NPE (harsh)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 9 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/docs/src/documentation/content/xdocs/hdfs_user_guide.xml

@@ -112,17 +112,18 @@
     		problems.
     	</li>
     	<li>
-    		Secondary NameNode (deprecated): performs periodic checkpoints of the 
+    		Secondary NameNode: performs periodic checkpoints of the 
     		namespace and helps keep the size of file containing log of HDFS 
     		modifications within certain limits at the NameNode.
-    		Replaced by Checkpoint node.
     	</li>
+
     	<li>
     		Checkpoint node: performs periodic checkpoints of the namespace and
     		helps minimize the size of the log stored at the NameNode 
     		containing changes to the HDFS.
-    		Replaces the role previously filled by the Secondary NameNode. 
-    		NameNode allows multiple Checkpoint nodes simultaneously, 
+    		Replaces the role previously filled by the Secondary NameNode,
+                though is not yet battle hardened.
+    		The NameNode allows multiple Checkpoint nodes simultaneously, 
     		as long as there are no Backup nodes registered with the system.
     	</li>
     	<li>
@@ -132,6 +133,7 @@
     		which is always in sync with the active NameNode namespace state.
     		Only one Backup node may be registered with the NameNode at once.
     	</li>
+
       </ul>
     </li>
     </ul>
@@ -234,12 +236,6 @@
    
    </section> 
 	<section> <title>Secondary NameNode</title>
-   <note>
-   The Secondary NameNode has been deprecated. 
-   Instead, consider using the 
-   <a href="hdfs_user_guide.html#Checkpoint+node">Checkpoint Node</a> or 
-   <a href="hdfs_user_guide.html#Backup+node">Backup Node</a>.
-   </note>
    <p>	
      The NameNode stores modifications to the file system as a log
      appended to a native file system file, <code>edits</code>. 
@@ -287,7 +283,9 @@
      <a href="http://hadoop.apache.org/common/docs/current/commands_manual.html#secondarynamenode">secondarynamenode</a>.
    </p>
    
-   </section><section> <title> Checkpoint Node </title>
+   </section>
+
+   <section> <title> Checkpoint Node </title>
    <p>NameNode persists its namespace using two files: <code>fsimage</code>,
       which is the latest checkpoint of the namespace and <code>edits</code>,
       a journal (log) of changes to the namespace since the checkpoint.

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -89,7 +89,6 @@ import com.google.common.collect.ImmutableList;
  * primary NameNode.
  *
  **********************************************************/
-@Deprecated // use BackupNode with -checkpoint argument instead.
 @InterfaceAudience.Private
 public class SecondaryNameNode implements Runnable {
     

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BenchmarkThroughput.java

@@ -193,6 +193,10 @@ public class BenchmarkThroughput extends Configured implements Tool {
     BUFFER_SIZE = conf.getInt("dfsthroughput.buffer.size", 4 * 1024);
 
     String localDir = conf.get("mapred.temp.dir");
+    if (localDir == null) {
+      localDir = conf.get("hadoop.tmp.dir");
+      conf.set("mapred.temp.dir", localDir);
+    }
     dir = new LocalDirAllocator("mapred.temp.dir");
 
     System.setProperty("test.build.data", localDir);

+ 0 - 26
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -205,7 +205,6 @@ public class TestCheckpoint extends TestCase {
   /*
    * Simulate namenode crashing after rolling edit log.
    */
-  @SuppressWarnings("deprecation")
   public void testSecondaryNamenodeError1()
     throws IOException {
     LOG.info("Starting testSecondaryNamenodeError1");
@@ -267,7 +266,6 @@ public class TestCheckpoint extends TestCase {
   /*
    * Simulate a namenode crash after uploading new image
    */
-  @SuppressWarnings("deprecation")
   public void testSecondaryNamenodeError2() throws IOException {
     LOG.info("Starting testSecondaryNamenodeError2");
     Configuration conf = new HdfsConfiguration();
@@ -326,7 +324,6 @@ public class TestCheckpoint extends TestCase {
   /*
    * Simulate a secondary namenode crash after rolling the edit log.
    */
-  @SuppressWarnings("deprecation")
   public void testSecondaryNamenodeError3() throws IOException {
     LOG.info("Starting testSecondaryNamenodeError3");
     Configuration conf = new HdfsConfiguration();
@@ -396,7 +393,6 @@ public class TestCheckpoint extends TestCase {
    * back to the name-node.
    * Used to truncate primary fsimage file.
    */
-  @SuppressWarnings("deprecation")
   public void testSecondaryFailsToReturnImage() throws IOException {
     LOG.info("Starting testSecondaryFailsToReturnImage");
     Configuration conf = new HdfsConfiguration();
@@ -473,7 +469,6 @@ public class TestCheckpoint extends TestCase {
    * @param errorType the ErrorSimulator type to trigger
    * @param exceptionSubstring an expected substring of the triggered exception
    */
-  @SuppressWarnings("deprecation")
   private void doSendFailTest(int errorType, String exceptionSubstring)
       throws IOException {
     Configuration conf = new HdfsConfiguration();
@@ -588,7 +583,6 @@ public class TestCheckpoint extends TestCase {
   /**
    * Test that the SecondaryNameNode properly locks its storage directories.
    */
-  @SuppressWarnings("deprecation")
   public void testSecondaryNameNodeLocking() throws Exception {
     // Start a primary NN so that the secondary will start successfully
     Configuration conf = new HdfsConfiguration();
@@ -681,7 +675,6 @@ public class TestCheckpoint extends TestCase {
    * 2. if the NN does not contain an image, importing a checkpoint
    *    succeeds and re-saves the image
    */
-  @SuppressWarnings("deprecation")
   public void testImportCheckpoint() throws Exception {
     Configuration conf = new HdfsConfiguration();
     Path testPath = new Path("/testfile");
@@ -762,16 +755,12 @@ public class TestCheckpoint extends TestCase {
       throw new IOException("Cannot create directory " + dir);
   }
   
-  // This deprecation suppress warning does not work due to known Java bug:
-  // http://bugs.sun.com/view_bug.do?bug_id=6460147
-  @SuppressWarnings("deprecation")
   SecondaryNameNode startSecondaryNameNode(Configuration conf
                                           ) throws IOException {
     conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     return new SecondaryNameNode(conf);
   }
   
-  @SuppressWarnings("deprecation")
   SecondaryNameNode startSecondaryNameNode(Configuration conf, int index)
       throws IOException {
     Configuration snnConf = new Configuration(conf);
@@ -784,7 +773,6 @@ public class TestCheckpoint extends TestCase {
   /**
    * Tests checkpoint in HDFS.
    */
-  @SuppressWarnings("deprecation")
   public void testCheckpoint() throws IOException {
     Path file1 = new Path("checkpoint.dat");
     Path file2 = new Path("checkpoint2.dat");
@@ -1015,7 +1003,6 @@ public class TestCheckpoint extends TestCase {
    * - it then fails again for the same reason
    * - it then tries to checkpoint a third time
    */
-  @SuppressWarnings("deprecation")
   public void testCheckpointAfterTwoFailedUploads() throws IOException {
     MiniDFSCluster cluster = null;
     SecondaryNameNode secondary = null;
@@ -1070,7 +1057,6 @@ public class TestCheckpoint extends TestCase {
    * 
    * @throws IOException
    */
-  @SuppressWarnings("deprecation")
   public void testMultipleSecondaryNamenodes() throws IOException {
     Configuration conf = new HdfsConfiguration();
     String nameserviceId1 = "ns1";
@@ -1121,7 +1107,6 @@ public class TestCheckpoint extends TestCase {
    * Test that the secondary doesn't have to re-download image
    * if it hasn't changed.
    */
-  @SuppressWarnings("deprecation")
   public void testSecondaryImageDownload() throws IOException {
     LOG.info("Starting testSecondaryImageDownload");
     Configuration conf = new HdfsConfiguration();
@@ -1204,7 +1189,6 @@ public class TestCheckpoint extends TestCase {
    * It verifies that this works even though the earlier-txid checkpoint gets
    * uploaded after the later-txid checkpoint.
    */
-  @SuppressWarnings("deprecation")
   public void testMultipleSecondaryNNsAgainstSameNN() throws Exception {
     Configuration conf = new HdfsConfiguration();
 
@@ -1290,7 +1274,6 @@ public class TestCheckpoint extends TestCase {
    * It verifies that one of the two gets an error that it's uploading a
    * duplicate checkpoint, and the other one succeeds.
    */
-  @SuppressWarnings("deprecation")
   public void testMultipleSecondaryNNsAgainstSameNN2() throws Exception {
     Configuration conf = new HdfsConfiguration();
 
@@ -1383,7 +1366,6 @@ public class TestCheckpoint extends TestCase {
    * is running. The secondary should shut itself down if if talks to a NN
    * with the wrong namespace.
    */
-  @SuppressWarnings("deprecation")
   public void testReformatNNBetweenCheckpoints() throws IOException {
     MiniDFSCluster cluster = null;
     SecondaryNameNode secondary = null;
@@ -1638,7 +1620,6 @@ public class TestCheckpoint extends TestCase {
   /**
    * Test that the 2NN triggers a checkpoint after the configurable interval
    */
-  @SuppressWarnings("deprecation")
   public void testCheckpointTriggerOnTxnCount() throws Exception {
     MiniDFSCluster cluster = null;
     SecondaryNameNode secondary = null;
@@ -1692,7 +1673,6 @@ public class TestCheckpoint extends TestCase {
    * logs that connect the 2NN's old checkpoint to the current txid
    * get archived. Then, the 2NN tries to checkpoint again.
    */
-  @SuppressWarnings("deprecation")
   public void testSecondaryHasVeryOutOfDateImage() throws IOException {
     MiniDFSCluster cluster = null;
     SecondaryNameNode secondary = null;
@@ -1730,7 +1710,6 @@ public class TestCheckpoint extends TestCase {
     }
   }
   
-  @SuppressWarnings("deprecation")
   public void testCommandLineParsing() throws ParseException {
     SecondaryNameNode.CommandLineOpts opts =
       new SecondaryNameNode.CommandLineOpts();
@@ -1765,7 +1744,6 @@ public class TestCheckpoint extends TestCase {
     } catch (ParseException e) {}
   }
 
-  @SuppressWarnings("deprecation")
   private void cleanup(SecondaryNameNode snn) {
     if (snn != null) {
       try {
@@ -1781,7 +1759,6 @@ public class TestCheckpoint extends TestCase {
    * Assert that if any two files have the same name across the 2NNs
    * and NN, they should have the same content too.
    */
-  @SuppressWarnings("deprecation")
   private void assertParallelFilesInvariant(MiniDFSCluster cluster,
       ImmutableList<SecondaryNameNode> secondaries) throws Exception {
     List<File> allCurrentDirs = Lists.newArrayList();
@@ -1793,7 +1770,6 @@ public class TestCheckpoint extends TestCase {
         ImmutableSet.of("VERSION"));    
   }
   
-  @SuppressWarnings("deprecation")
   private List<File> getCheckpointCurrentDirs(SecondaryNameNode secondary) {
     List<File> ret = Lists.newArrayList();
     for (URI u : secondary.getCheckpointDirs()) {
@@ -1803,7 +1779,6 @@ public class TestCheckpoint extends TestCase {
     return ret;
   }
 
-  @SuppressWarnings("deprecation")
   private CheckpointStorage spyOnSecondaryImage(SecondaryNameNode secondary1) {
     CheckpointStorage spy = Mockito.spy((CheckpointStorage)secondary1.getFSImage());;
     secondary1.setFSImage(spy);
@@ -1813,7 +1788,6 @@ public class TestCheckpoint extends TestCase {
   /**
    * A utility class to perform a checkpoint in a different thread.
    */
-  @SuppressWarnings("deprecation")
   private static class DoCheckpointThread extends Thread {
     private final SecondaryNameNode snn;
     private volatile Throwable thrown = null;

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java

@@ -120,9 +120,6 @@ public class TestNameEditsConfigs {
     assertTrue(!fileSys.exists(name));
   }
 
-  // This deprecation suppress warning does not work due to known Java bug:
-  // http://bugs.sun.com/view_bug.do?bug_id=6460147
-  @SuppressWarnings("deprecation")
   SecondaryNameNode startSecondaryNameNode(Configuration conf
                                           ) throws IOException {
     conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, "0.0.0.0:0");
@@ -142,7 +139,6 @@ public class TestNameEditsConfigs {
    * sure we are reading proper edits and image.
    * @throws Exception 
    */
-  @SuppressWarnings("deprecation")
   @Test
   public void testNameEditsConfigs() throws Exception {
     Path file1 = new Path("TestNameEditsConfigs1");

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.java

@@ -30,7 +30,6 @@ import org.junit.Test;
 
 public class TestSecondaryWebUi {
 
-  @SuppressWarnings("deprecation")
   @Test
   public void testSecondaryWebUi() throws IOException {
     Configuration conf = new Configuration();

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java

@@ -121,7 +121,6 @@ public class TestStartup extends TestCase {
    * start MiniDFScluster, create a file (to create edits) and do a checkpoint  
    * @throws IOException
    */
-  @SuppressWarnings("deprecation")
   public void createCheckPoint() throws IOException {
     LOG.info("--starting mini cluster");
     // manage dirs parameter set to false 
@@ -303,7 +302,6 @@ public class TestStartup extends TestCase {
    * secondary node copies fsimage and edits into correct separate directories.
    * @throws IOException
    */
-  @SuppressWarnings("deprecation")
   public void testSNNStartup() throws IOException{
     //setUpConfig();
     LOG.info("--starting SecondNN startup test");

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java

@@ -155,7 +155,6 @@ public class TestStorageRestore {
    * 7. run doCheckpoint
    * 8. verify that all the image and edits files are the same.
    */
-  @SuppressWarnings("deprecation")
   @Test
   public void testStorageRestore() throws Exception {
     int numDatanodes = 0;
@@ -312,7 +311,6 @@ public class TestStorageRestore {
    * then try to perform a checkpoint. The NN should not serve up the image or
    * edits from the restored (empty) dir.
    */
-  @SuppressWarnings("deprecation")
   @Test
   public void testMultipleSecondaryCheckpoint() throws IOException {
     

+ 2 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -538,6 +538,8 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3646. Remove redundant URL info from "mapred job" output.
     (Jonathan Eagles via mahadev)
 
+    MAPREDUCE-3681. Fixed computation of queue's usedCapacity. (acmurthy) 
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 14 - 9
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -272,9 +272,9 @@ public class LeafQueue implements CSQueue {
         "maxActiveApplicationsPerUser = " + maxActiveApplicationsPerUser +
         " [= (int)(maxActiveApplications * (userLimit / 100.0f) * userLimitFactor) ]" + "\n" +
         "utilization = " + utilization +
-        " [= usedResourcesMemory / queueLimit ]" + "\n" +
+        " [= usedResourcesMemory /  (clusterResourceMemory * absoluteCapacity)]" + "\n" +
         "usedCapacity = " + usedCapacity +
-        " [= usedResourcesMemory / (clusterResourceMemory * capacity) ]" + "\n" +
+        " [= usedResourcesMemory / (clusterResourceMemory * parent.absoluteCapacity)]" + "\n" +
         "maxAMResourcePercent = " + maxAMResourcePercent +
         " [= configuredMaximumAMResourcePercent ]" + "\n" +
         "minimumAllocationFactor = " + minimumAllocationFactor +
@@ -502,9 +502,14 @@ public class LeafQueue implements CSQueue {
   }
 
   public String toString() {
-    return queueName + ":" + capacity + ":" + absoluteCapacity + ":" + 
-    getUsedCapacity() + ":" + getUtilization() + ":" + 
-    getNumApplications() + ":" + getNumContainers();
+    return queueName + ": " + 
+        "capacity=" + capacity + ", " + 
+        "absoluteCapacity=" + absoluteCapacity + ", " + 
+        "usedResources=" + usedResources.getMemory() + "MB, " + 
+        "usedCapacity=" + getUsedCapacity() + ", " + 
+        "utilization=" + getUtilization() + ", " + 
+        "numApps=" + getNumApplications() + ", " + 
+        "numContainers=" + getNumContainers();  
   }
 
   private synchronized User getUser(String userName) {
@@ -1316,11 +1321,11 @@ public class LeafQueue implements CSQueue {
   }
   
   private synchronized void updateResource(Resource clusterResource) {
-    float queueLimit = clusterResource.getMemory() * absoluteCapacity; 
+    float queueLimit = clusterResource.getMemory() * absoluteCapacity;
     setUtilization(usedResources.getMemory() / queueLimit);
-    setUsedCapacity(
-        usedResources.getMemory() / (clusterResource.getMemory() * capacity));
-    
+    setUsedCapacity(usedResources.getMemory()
+        / (clusterResource.getMemory() * parent.getAbsoluteCapacity()));
+
     Resource resourceLimit = 
       Resources.createResource(roundUp((int)queueLimit));
     metrics.setAvailableResourcesToQueue(

+ 15 - 8
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java

@@ -333,10 +333,15 @@ public class ParentQueue implements CSQueue {
   }
 
   public String toString() {
-    return queueName + ":" + capacity + ":" + absoluteCapacity + ":" + 
-      getUsedCapacity() + ":" + getUtilization() + ":" + 
-      getNumApplications() + ":" + getNumContainers() + ":" + 
-      childQueues.size() + " child-queues";
+    return queueName + ": " +
+        "numChildQueue= " + childQueues.size() + ", " + 
+        "capacity=" + capacity + ", " +  
+        "absoluteCapacity=" + absoluteCapacity + ", " +
+        "usedResources=" + usedResources.getMemory() + "MB, " + 
+        "usedCapacity=" + getUsedCapacity() + ", " + 
+        "utilization=" + getUtilization() + ", " +
+        "numApps=" + getNumApplications() + ", " + 
+        "numContainers=" + getNumContainers();
   }
   
   @Override
@@ -688,11 +693,13 @@ public class ParentQueue implements CSQueue {
   }
   
   private synchronized void updateResource(Resource clusterResource) {
-    float queueLimit = clusterResource.getMemory() * absoluteCapacity; 
+    float queueLimit = clusterResource.getMemory() * absoluteCapacity;
+    float parentAbsoluteCapacity = 
+        (rootQueue) ? 1.0f : parent.getAbsoluteCapacity();
     setUtilization(usedResources.getMemory() / queueLimit);
-    setUsedCapacity(
-        usedResources.getMemory() / (clusterResource.getMemory() * capacity));
-    
+    setUsedCapacity(usedResources.getMemory() 
+        / (clusterResource.getMemory() * parentAbsoluteCapacity));
+  
     Resource resourceLimit = 
       Resources.createResource((int)queueLimit);
     metrics.setAvailableResourcesToQueue(

+ 52 - 62
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java

@@ -138,12 +138,34 @@ public class TestParentQueue {
     when(queue).assignContainers(eq(clusterResource), eq(node));
   }
   
+  private float computeQueueUsedCapacity(CSQueue queue, 
+      int expectedMemory, Resource clusterResource) {
+    return (
+        ((float)expectedMemory / clusterResource.getMemory()) *
+        queue.getParent().getAbsoluteCapacity()
+      );
+  }
+  
   private float computeQueueUtilization(CSQueue queue, 
       int expectedMemory, Resource clusterResource) {
     return (expectedMemory / 
         (clusterResource.getMemory() * queue.getAbsoluteCapacity()));
   }
   
+  final static float DELTA = 0.0001f;
+  private void verifyQueueMetrics(CSQueue queue, 
+      int expectedMemory, Resource clusterResource) {
+    assertEquals(
+        computeQueueUtilization(queue, expectedMemory, clusterResource), 
+        queue.getUtilization(), 
+        DELTA);
+    assertEquals(
+        computeQueueUsedCapacity(queue, expectedMemory, clusterResource), 
+        queue.getUsedCapacity(), 
+        DELTA);
+
+  }
+  
   @Test
   public void testSingleLevelQueues() throws Exception {
     // Setup queue configs
@@ -173,15 +195,13 @@ public class TestParentQueue {
     // Start testing
     LeafQueue a = (LeafQueue)queues.get(A);
     LeafQueue b = (LeafQueue)queues.get(B);
-    final float delta = 0.0001f;
     
     // Simulate B returning a container on node_0
     stubQueueAllocation(a, clusterResource, node_0, 0*GB);
     stubQueueAllocation(b, clusterResource, node_0, 1*GB);
     root.assignContainers(clusterResource, node_0);
-    assertEquals(0.0f, a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 1*GB, clusterResource), 
-        b.getUtilization(), delta);
+    verifyQueueMetrics(a, 0*GB, clusterResource);
+    verifyQueueMetrics(b, 1*GB, clusterResource);
     
     // Now, A should get the scheduling opportunity since A=0G/6G, B=1G/14G
     stubQueueAllocation(a, clusterResource, node_1, 2*GB);
@@ -192,10 +212,8 @@ public class TestParentQueue {
         any(SchedulerNode.class));
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
         any(SchedulerNode.class));
-    assertEquals(computeQueueUtilization(a, 2*GB, clusterResource), 
-        a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 2*GB, clusterResource), 
-        b.getUtilization(), delta);
+    verifyQueueMetrics(a, 2*GB, clusterResource);
+    verifyQueueMetrics(b, 2*GB, clusterResource);
 
     // Now, B should get the scheduling opportunity 
     // since A has 2/6G while B has 2/14G
@@ -207,10 +225,8 @@ public class TestParentQueue {
         any(SchedulerNode.class));
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
         any(SchedulerNode.class));
-    assertEquals(computeQueueUtilization(a, 3*GB, clusterResource), 
-        a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 4*GB, clusterResource), 
-        b.getUtilization(), delta);
+    verifyQueueMetrics(a, 3*GB, clusterResource);
+    verifyQueueMetrics(b, 4*GB, clusterResource);
 
     // Now, B should still get the scheduling opportunity 
     // since A has 3/6G while B has 4/14G
@@ -222,10 +238,8 @@ public class TestParentQueue {
         any(SchedulerNode.class));
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
         any(SchedulerNode.class));
-    assertEquals(computeQueueUtilization(a, 3*GB, clusterResource), 
-        a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 8*GB, clusterResource), 
-        b.getUtilization(), delta);
+    verifyQueueMetrics(a, 3*GB, clusterResource);
+    verifyQueueMetrics(b, 8*GB, clusterResource);
 
     // Now, A should get the scheduling opportunity 
     // since A has 3/6G while B has 8/14G
@@ -237,10 +251,8 @@ public class TestParentQueue {
         any(SchedulerNode.class));
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
         any(SchedulerNode.class));
-    assertEquals(computeQueueUtilization(a, 4*GB, clusterResource), 
-        a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 9*GB, clusterResource), 
-        b.getUtilization(), delta);
+    verifyQueueMetrics(a, 4*GB, clusterResource);
+    verifyQueueMetrics(b, 9*GB, clusterResource);
   }
 
   private static final String C = "c";
@@ -323,22 +335,16 @@ public class TestParentQueue {
     CSQueue b2 = queues.get(B2);
     CSQueue b3 = queues.get(B3);
 
-    final float delta = 0.0001f;
-    
     // Simulate C returning a container on node_0
     stubQueueAllocation(a, clusterResource, node_0, 0*GB);
     stubQueueAllocation(b, clusterResource, node_0, 0*GB);
     stubQueueAllocation(c, clusterResource, node_0, 1*GB);
     stubQueueAllocation(d, clusterResource, node_0, 0*GB);
     root.assignContainers(clusterResource, node_0);
-    assertEquals(computeQueueUtilization(a, 0*GB, clusterResource), 
-        a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 0*GB, clusterResource), 
-        b.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(c, 1*GB, clusterResource), 
-        c.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(d, 0*GB, clusterResource), 
-        d.getUtilization(), delta);
+    verifyQueueMetrics(a, 0*GB, clusterResource);
+    verifyQueueMetrics(b, 0*GB, clusterResource);
+    verifyQueueMetrics(c, 1*GB, clusterResource);
+    verifyQueueMetrics(d, 0*GB, clusterResource);
     reset(a); reset(b); reset(c);
 
     // Now get B2 to allocate
@@ -347,12 +353,9 @@ public class TestParentQueue {
     stubQueueAllocation(b2, clusterResource, node_1, 4*GB);
     stubQueueAllocation(c, clusterResource, node_1, 0*GB);
     root.assignContainers(clusterResource, node_1);
-    assertEquals(computeQueueUtilization(a, 0*GB, clusterResource), 
-        a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 4*GB, clusterResource), 
-        b.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(c, 1*GB, clusterResource), 
-        c.getUtilization(), delta);
+    verifyQueueMetrics(a, 0*GB, clusterResource);
+    verifyQueueMetrics(b, 4*GB, clusterResource);
+    verifyQueueMetrics(c, 1*GB, clusterResource);
     reset(a); reset(b); reset(c);
     
     // Now get both A1, C & B3 to allocate in right order
@@ -368,12 +371,9 @@ public class TestParentQueue {
         any(SchedulerNode.class));
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
         any(SchedulerNode.class));
-    assertEquals(computeQueueUtilization(a, 1*GB, clusterResource), 
-        a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 6*GB, clusterResource), 
-        b.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(c, 3*GB, clusterResource), 
-        c.getUtilization(), delta);
+    verifyQueueMetrics(a, 1*GB, clusterResource);
+    verifyQueueMetrics(b, 6*GB, clusterResource);
+    verifyQueueMetrics(c, 3*GB, clusterResource);
     reset(a); reset(b); reset(c);
     
     // Now verify max-capacity
@@ -399,16 +399,12 @@ public class TestParentQueue {
         any(SchedulerNode.class));
     allocationOrder.verify(c).assignContainers(eq(clusterResource), 
         any(SchedulerNode.class));
-    assertEquals(computeQueueUtilization(a, 3*GB, clusterResource), 
-        a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 8*GB, clusterResource), 
-        b.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(c, 4*GB, clusterResource), 
-        c.getUtilization(), delta);
+    verifyQueueMetrics(a, 3*GB, clusterResource);
+    verifyQueueMetrics(b, 8*GB, clusterResource);
+    verifyQueueMetrics(c, 4*GB, clusterResource);
     reset(a); reset(b); reset(c);
-    
   }
-  
+
   @Test
   public void testOffSwitchScheduling() throws Exception {
     // Setup queue configs
@@ -438,15 +434,13 @@ public class TestParentQueue {
     // Start testing
     LeafQueue a = (LeafQueue)queues.get(A);
     LeafQueue b = (LeafQueue)queues.get(B);
-    final float delta = 0.0001f;
     
     // Simulate B returning a container on node_0
     stubQueueAllocation(a, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH);
     stubQueueAllocation(b, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
     root.assignContainers(clusterResource, node_0);
-    assertEquals(0.0f, a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 1*GB, clusterResource), 
-        b.getUtilization(), delta);
+    verifyQueueMetrics(a, 0*GB, clusterResource);
+    verifyQueueMetrics(b, 1*GB, clusterResource);
     
     // Now, A should get the scheduling opportunity since A=0G/6G, B=1G/14G
     // also, B gets a scheduling opportunity since A allocates RACK_LOCAL
@@ -458,10 +452,8 @@ public class TestParentQueue {
         any(SchedulerNode.class));
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
         any(SchedulerNode.class));
-    assertEquals(computeQueueUtilization(a, 2*GB, clusterResource), 
-        a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 2*GB, clusterResource), 
-        b.getUtilization(), delta);
+    verifyQueueMetrics(a, 2*GB, clusterResource);
+    verifyQueueMetrics(b, 2*GB, clusterResource);
     
     // Now, B should get the scheduling opportunity 
     // since A has 2/6G while B has 2/14G, 
@@ -474,10 +466,8 @@ public class TestParentQueue {
         any(SchedulerNode.class));
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
         any(SchedulerNode.class));
-    assertEquals(computeQueueUtilization(a, 2*GB, clusterResource), 
-        a.getUtilization(), delta);
-    assertEquals(computeQueueUtilization(b, 4*GB, clusterResource), 
-        b.getUtilization(), delta);
+    verifyQueueMetrics(a, 2*GB, clusterResource);
+    verifyQueueMetrics(b, 4*GB, clusterResource);
 
   }
   

+ 10 - 10
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm

@@ -95,7 +95,7 @@ Hadoop MapReduce Next Generation - Cluster Setup
 *--------------------------------------+--------------------------------------+
 | DataNode                             | HADOOP_DATANODE_OPTS                 |
 *--------------------------------------+--------------------------------------+
-| Backup NameNode                      | HADOOP_SECONDARYNAMENODE_OPTS        |
+| Secondary NameNode                   | HADOOP_SECONDARYNAMENODE_OPTS        |
 *--------------------------------------+--------------------------------------+
 | ResourceManager                      | YARN_RESOURCEMANAGER_OPTS            |
 *--------------------------------------+--------------------------------------+
@@ -537,15 +537,15 @@ Hadoop MapReduce Next Generation - Cluster Setup
       
   It's recommended to have them share a Unix group, for e.g. <<<hadoop>>>.
       
-*--------------------------------------+--------------------------------------+
-|| User:Group                          || Daemons                             |
-*--------------------------------------+--------------------------------------+
-| hdfs:hadoop                          | NameNode, Backup NameNode, DataNode  |
-*--------------------------------------+--------------------------------------+
-| yarn:hadoop                          | ResourceManager, NodeManager         |
-*--------------------------------------+--------------------------------------+
-| mapred:hadoop                        | MapReduce JobHistory Server          |  
-*--------------------------------------+--------------------------------------+
+*--------------------------------------+----------------------------------------------------------------------+
+|| User:Group                          || Daemons                                                             |
+*--------------------------------------+----------------------------------------------------------------------+
+| hdfs:hadoop                          | NameNode, Secondary NameNode, Checkpoint Node, Backup Node, DataNode |
+*--------------------------------------+----------------------------------------------------------------------+
+| yarn:hadoop                          | ResourceManager, NodeManager                                         |
+*--------------------------------------+----------------------------------------------------------------------+
+| mapred:hadoop                        | MapReduce JobHistory Server                                          |
+*--------------------------------------+----------------------------------------------------------------------+
       
   * <<<Permissions for both HDFS and local fileSystem paths>>>