Browse Source

MAPREDUCE-3812. Lower default allocation sizes, fix allocation configurations and document them (Harsh J via bobby)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1329282 13f79535-47bb-0310-9956-ffa450edef68
Robert Joseph Evans 13 years ago
parent
commit
1eb1d0ae4f
13 changed files with 80 additions and 56 deletions
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 2 13
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
  3. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  4. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  5. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml
  6. 2 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  7. 11 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  8. 16 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  9. 7 14
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
  10. 7 15
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  11. 13 6
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java
  12. 7 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
  13. 8 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm

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

@@ -262,6 +262,9 @@ Release 0.23.3 - UNRELEASED
     MAPREDUCE-4072. User set java.library.path seems to overwrite default
     MAPREDUCE-4072. User set java.library.path seems to overwrite default
     creating problems native lib loading (Anupam Seth via bobby)
     creating problems native lib loading (Anupam Seth via bobby)
 
 
+    MAPREDUCE-3812. Lower default allocation sizes, fix allocation 
+    configurations and document them (Harsh J via bobby)
+
   NEW FEATURES
   NEW FEATURES
 
 
   IMPROVEMENTS
   IMPROVEMENTS

+ 2 - 13
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java

@@ -65,6 +65,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.Event;
@@ -341,7 +342,7 @@ public class TestRMContainerAllocator {
     }
     }
     @Override
     @Override
     protected ResourceScheduler createScheduler() {
     protected ResourceScheduler createScheduler() {
-      return new MyFifoScheduler(getRMContext());
+      return new MyFifoScheduler();
     }
     }
   }
   }
 
 
@@ -997,18 +998,6 @@ public class TestRMContainerAllocator {
   }
   }
   
   
   private static class MyFifoScheduler extends FifoScheduler {
   private static class MyFifoScheduler extends FifoScheduler {
-
-    public MyFifoScheduler(RMContext rmContext) {
-      super();
-      try {
-        reinitialize(new Configuration(), new ContainerTokenSecretManager(),
-            rmContext);
-      } catch (IOException ie) {
-        LOG.info("add application failed with ", ie);
-        assert (false);
-      }
-    }
-
     // override this to copy the objects otherwise FifoScheduler updates the
     // override this to copy the objects otherwise FifoScheduler updates the
     // numContainers in same objects as kept by RMContainerAllocator
     // numContainers in same objects as kept by RMContainerAllocator
     @Override
     @Override

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java

@@ -341,12 +341,12 @@ public interface MRJobConfig {
   /** The amount of memory the MR app master needs.*/
   /** The amount of memory the MR app master needs.*/
   public static final String MR_AM_VMEM_MB =
   public static final String MR_AM_VMEM_MB =
     MR_AM_PREFIX+"resource.mb";
     MR_AM_PREFIX+"resource.mb";
-  public static final int DEFAULT_MR_AM_VMEM_MB = 2048;
+  public static final int DEFAULT_MR_AM_VMEM_MB = 1536;
 
 
   /** Command line arguments passed to the MR app master.*/
   /** Command line arguments passed to the MR app master.*/
   public static final String MR_AM_COMMAND_OPTS =
   public static final String MR_AM_COMMAND_OPTS =
     MR_AM_PREFIX+"command-opts";
     MR_AM_PREFIX+"command-opts";
-  public static final String DEFAULT_MR_AM_COMMAND_OPTS = "-Xmx1536m";
+  public static final String DEFAULT_MR_AM_COMMAND_OPTS = "-Xmx1024m";
 
 
   /** Root Logging level passed to the MR app master.*/
   /** Root Logging level passed to the MR app master.*/
   public static final String MR_AM_LOG_LEVEL = 
   public static final String MR_AM_LOG_LEVEL = 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -1259,7 +1259,7 @@
 
 
 <property>
 <property>
   <name>yarn.app.mapreduce.am.resource.mb</name>
   <name>yarn.app.mapreduce.am.resource.mb</name>
-  <value>2048</value>
+  <value>1536</value>
   <description>The amount of memory the MR AppMaster needs.</description>
   <description>The amount of memory the MR AppMaster needs.</description>
 </property>
 </property>
 
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/resources/job_1329348432655_0001_conf.xml

@@ -292,7 +292,7 @@
 <property><!--Loaded from job.xml--><name>yarn.resourcemanager.scheduler.address</name><value>0.0.0.0:8030</value></property>
 <property><!--Loaded from job.xml--><name>yarn.resourcemanager.scheduler.address</name><value>0.0.0.0:8030</value></property>
 <property><!--Loaded from job.xml--><name>fs.trash.checkpoint.interval</name><value>0</value></property>
 <property><!--Loaded from job.xml--><name>fs.trash.checkpoint.interval</name><value>0</value></property>
 <property><!--Loaded from job.xml--><name>s3native.stream-buffer-size</name><value>4096</value></property>
 <property><!--Loaded from job.xml--><name>s3native.stream-buffer-size</name><value>4096</value></property>
-<property><!--Loaded from job.xml--><name>yarn.scheduler.fifo.minimum-allocation-mb</name><value>1024</value></property>
+<property><!--Loaded from job.xml--><name>yarn.scheduler.minimum-allocation-mb</name><value>128</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.reduce.shuffle.read.timeout</name><value>180000</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.reduce.shuffle.read.timeout</name><value>180000</value></property>
 <property><!--Loaded from job.xml--><name>yarn.app.mapreduce.am.command-opts</name><value>-Xmx500m</value></property>
 <property><!--Loaded from job.xml--><name>yarn.app.mapreduce.am.command-opts</name><value>-Xmx500m</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.admin.user.env</name><value>LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native</value></property>
 <property><!--Loaded from job.xml--><name>mapreduce.admin.user.env</name><value>LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native</value></property>

+ 2 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.JarFinder;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
@@ -48,7 +49,7 @@ public class TestDistributedShell {
   @BeforeClass
   @BeforeClass
   public static void setup() throws InterruptedException, IOException {
   public static void setup() throws InterruptedException, IOException {
     LOG.info("Starting up YARN cluster");
     LOG.info("Starting up YARN cluster");
-    conf.setInt("yarn.scheduler.fifo.minimum-allocation-mb", 128);
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128);
     if (yarnCluster == null) {
     if (yarnCluster == null) {
       yarnCluster = new MiniYARNCluster(TestDistributedShell.class.getName(),
       yarnCluster = new MiniYARNCluster(TestDistributedShell.class.getName(),
           1, 1, 1);
           1, 1, 1);

+ 11 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -107,7 +107,17 @@ public class YarnConfiguration extends Configuration {
   public static final int DEFAULT_RM_SCHEDULER_PORT = 8030;
   public static final int DEFAULT_RM_SCHEDULER_PORT = 8030;
   public static final String DEFAULT_RM_SCHEDULER_ADDRESS = "0.0.0.0:" +
   public static final String DEFAULT_RM_SCHEDULER_ADDRESS = "0.0.0.0:" +
     DEFAULT_RM_SCHEDULER_PORT;
     DEFAULT_RM_SCHEDULER_PORT;
-  
+
+  /** Miniumum memory request grant-able by the RM scheduler. */
+  public static final String RM_SCHEDULER_MINIMUM_ALLOCATION_MB =
+    YARN_PREFIX + "scheduler.minimum-allocation-mb";
+  public static final int DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB = 128;
+
+  /** Maximum memory request grant-able by the RM scheduler. */
+  public static final String RM_SCHEDULER_MAXIMUM_ALLOCATION_MB =
+    YARN_PREFIX + "scheduler.maximum-allocation-mb";
+  public static final int DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB = 10240;
+
   /** Number of threads to handle scheduler interface.*/
   /** Number of threads to handle scheduler interface.*/
   public static final String RM_SCHEDULER_CLIENT_THREAD_COUNT =
   public static final String RM_SCHEDULER_CLIENT_THREAD_COUNT =
     RM_PREFIX + "scheduler.client.thread-count";
     RM_PREFIX + "scheduler.client.thread-count";

+ 16 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -187,6 +187,22 @@
     <name>yarn.resourcemanager.scheduler.class</name>
     <name>yarn.resourcemanager.scheduler.class</name>
   </property>
   </property>
 
 
+  <property>
+    <description>The minimum allocation size for every container request at the RM,
+    in MBs. Memory requests lower than this won't take effect,
+    and the specified value will get allocated at minimum.</description>
+    <name>yarn.scheduler.minimum-allocation-mb</name>
+    <value>128</value>
+  </property>
+
+  <property>
+    <description>The maximum allocation size for every container request at the RM,
+    in MBs. Memory requests higher than this won't take effect,
+    and will get capped to this value.</description>
+    <name>yarn.scheduler.maximum-allocation-mb</name>
+    <value>10240</value>
+  </property>
+
   <property>
   <property>
     <description>The class to use as the persistent store.</description>
     <description>The class to use as the persistent store.</description>
     <name>yarn.resourcemanager.store.class</name>
     <name>yarn.resourcemanager.store.class</name>

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

@@ -30,6 +30,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 
 public class CapacitySchedulerConfiguration extends Configuration {
 public class CapacitySchedulerConfiguration extends Configuration {
@@ -71,18 +72,6 @@ public class CapacitySchedulerConfiguration extends Configuration {
   @Private
   @Private
   public static final String STATE = "state";
   public static final String STATE = "state";
 
 
-  private static final int MINIMUM_MEMORY = 1024;
-
-  @Private
-  public static final String MINIMUM_ALLOCATION = 
-    PREFIX + "minimum-allocation-mb";
-
-  private static final int MAXIMUM_MEMORY = 10240;
-
-  @Private
-  public static final String MAXIMUM_ALLOCATION = 
-    PREFIX + "maximum-allocation-mb";
-
   @Private
   @Private
   public static final int DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS = 10000;
   public static final int DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS = 10000;
   
   
@@ -253,12 +242,16 @@ public class CapacitySchedulerConfiguration extends Configuration {
   }
   }
   
   
   public Resource getMinimumAllocation() {
   public Resource getMinimumAllocation() {
-    int minimumMemory = getInt(MINIMUM_ALLOCATION, MINIMUM_MEMORY);
+    int minimumMemory = getInt(
+        YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
     return Resources.createResource(minimumMemory);
     return Resources.createResource(minimumMemory);
   }
   }
 
 
   public Resource getMaximumAllocation() {
   public Resource getMaximumAllocation() {
-    int maximumMemory = getInt(MAXIMUM_ALLOCATION, MAXIMUM_MEMORY);
+    int maximumMemory = getInt(
+        YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
     return Resources.createResource(maximumMemory);
     return Resources.createResource(maximumMemory);
   }
   }
 
 

+ 7 - 15
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
@@ -107,19 +108,6 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
 
 
   private Map<NodeId, SchedulerNode> nodes = new ConcurrentHashMap<NodeId, SchedulerNode>();
   private Map<NodeId, SchedulerNode> nodes = new ConcurrentHashMap<NodeId, SchedulerNode>();
 
 
-  private static final int MINIMUM_MEMORY = 1024;
-
-  private static final String FIFO_PREFIX =  "yarn.scheduler.fifo.";
-  @Private
-  public static final String MINIMUM_ALLOCATION = 
-    FIFO_PREFIX + "minimum-allocation-mb";
-
-  private static final int MAXIMUM_MEMORY = 10240;
-
-  @Private
-  public static final String MAXIMUM_ALLOCATION = 
-    FIFO_PREFIX + "maximum-allocation-mb";
-
   private boolean initialized;
   private boolean initialized;
   private Resource minimumAllocation;
   private Resource minimumAllocation;
   private Resource maximumAllocation;
   private Resource maximumAllocation;
@@ -218,9 +206,13 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
       this.containerTokenSecretManager = containerTokenSecretManager;
       this.containerTokenSecretManager = containerTokenSecretManager;
       this.rmContext = rmContext;
       this.rmContext = rmContext;
       this.minimumAllocation = 
       this.minimumAllocation = 
-        Resources.createResource(conf.getInt(MINIMUM_ALLOCATION, MINIMUM_MEMORY));
+        Resources.createResource(conf.getInt(
+            YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB));
       this.maximumAllocation = 
       this.maximumAllocation = 
-        Resources.createResource(conf.getInt(MAXIMUM_ALLOCATION, MAXIMUM_MEMORY));
+        Resources.createResource(conf.getInt(
+            YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
       this.metrics = QueueMetrics.forQueue(DEFAULT_QUEUE_NAME, null, false,
       this.metrics = QueueMetrics.forQueue(DEFAULT_QUEUE_NAME, null, false,
           conf);
           conf);
       this.activeUsersManager = new ActiveUsersManager(metrics);
       this.activeUsersManager = new ActiveUsersManager(metrics);

+ 13 - 6
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java

@@ -137,7 +137,7 @@ public class TestFifoScheduler {
     rm.stop();
     rm.stop();
   }
   }
 
 
-  private void testMinimumAllocation(YarnConfiguration conf)
+  private void testMinimumAllocation(YarnConfiguration conf, int testAlloc)
       throws Exception {
       throws Exception {
     MockRM rm = new MockRM(conf);
     MockRM rm = new MockRM(conf);
     rm.start();
     rm.start();
@@ -146,7 +146,7 @@ public class TestFifoScheduler {
     MockNM nm1 = rm.registerNode("h1:1234", 6 * GB);
     MockNM nm1 = rm.registerNode("h1:1234", 6 * GB);
 
 
     // Submit an application
     // Submit an application
-    RMApp app1 = rm.submitApp(256);
+    RMApp app1 = rm.submitApp(testAlloc);
 
 
     // kick the scheduling
     // kick the scheduling
     nm1.nodeHeartbeat(true);
     nm1.nodeHeartbeat(true);
@@ -157,7 +157,8 @@ public class TestFifoScheduler {
         nm1.getNodeId());
         nm1.getNodeId());
 
 
     int checkAlloc =
     int checkAlloc =
-        conf.getInt("yarn.scheduler.fifo.minimum-allocation-mb", GB);
+        conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
     Assert.assertEquals(checkAlloc, report_nm1.getUsedResource().getMemory());
     Assert.assertEquals(checkAlloc, report_nm1.getUsedResource().getMemory());
 
 
     rm.stop();
     rm.stop();
@@ -165,14 +166,20 @@ public class TestFifoScheduler {
 
 
   @Test
   @Test
   public void testDefaultMinimumAllocation() throws Exception {
   public void testDefaultMinimumAllocation() throws Exception {
-    testMinimumAllocation(new YarnConfiguration());
+    // Test with something lesser than default
+    testMinimumAllocation(
+        new YarnConfiguration(),
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB / 2);
   }
   }
 
 
   @Test
   @Test
   public void testNonDefaultMinimumAllocation() throws Exception {
   public void testNonDefaultMinimumAllocation() throws Exception {
+    // Set custom min-alloc to test tweaking it
+    int allocMB = 512;
     YarnConfiguration conf = new YarnConfiguration();
     YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt("yarn.scheduler.fifo.minimum-allocation-mb", 512);
-    testMinimumAllocation(conf);
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, allocMB);
+    // Test for something lesser than this.
+    testMinimumAllocation(conf, allocMB / 2);
   }
   }
 
 
   @Test
   @Test

+ 7 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java

@@ -31,6 +31,7 @@ import javax.xml.parsers.DocumentBuilderFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueState;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -540,8 +541,12 @@ public class TestRMWebServices extends JerseyTest {
     assertEquals("qstate doesn't match", QueueState.RUNNING.toString(), state);
     assertEquals("qstate doesn't match", QueueState.RUNNING.toString(), state);
     assertEquals("capacity doesn't match", 1.0, capacity, 0.0);
     assertEquals("capacity doesn't match", 1.0, capacity, 0.0);
     assertEquals("usedCapacity doesn't match", 0.0, usedCapacity, 0.0);
     assertEquals("usedCapacity doesn't match", 0.0, usedCapacity, 0.0);
-    assertEquals("minQueueMemoryCapacity doesn't match", 1024, minQueueCapacity);
-    assertEquals("maxQueueMemoryCapacity doesn't match", 10240,
+    assertEquals(
+        "minQueueMemoryCapacity doesn't match",
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+        minQueueCapacity);
+    assertEquals("maxQueueMemoryCapacity doesn't match",
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
         maxQueueCapacity);
         maxQueueCapacity);
     assertEquals("numNodes doesn't match", 0, numNodes);
     assertEquals("numNodes doesn't match", 0, numNodes);
     assertEquals("usedNodeCapacity doesn't match", 0, usedNodeCapacity);
     assertEquals("usedNodeCapacity doesn't match", 0, usedNodeCapacity);

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

@@ -245,6 +245,14 @@ Hadoop MapReduce Next Generation - Cluster Setup
 | | <<<ResourceManager>>> Scheduler class. | |
 | | <<<ResourceManager>>> Scheduler class. | |
 | | | <<<CapacityScheduler>>> (recommended) or <<<FifoScheduler>>> |
 | | | <<<CapacityScheduler>>> (recommended) or <<<FifoScheduler>>> |
 *-------------------------+-------------------------+------------------------+
 *-------------------------+-------------------------+------------------------+
+| <<<yarn.scheduler.minimum-allocation-mb>>> | | |
+| | Minimum limit of memory to allocate to each container request at the <<<Resource Manager>>>. | |
+| | | In MBs |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.scheduler.maximum-allocation-mb>>> | | |
+| | Maximum limit of memory to allocate to each container request at the <<<Resource Manager>>>. | |
+| | | In MBs |
+*-------------------------+-------------------------+------------------------+
 | <<<yarn.resourcemanager.nodes.include-path>>> / | | | 
 | <<<yarn.resourcemanager.nodes.include-path>>> / | | | 
 | <<<yarn.resourcemanager.nodes.exclude-path>>> | | |  
 | <<<yarn.resourcemanager.nodes.exclude-path>>> | | |  
 | | List of permitted/excluded NodeManagers. | |
 | | List of permitted/excluded NodeManagers. | |