Merge -c 1189542 from trunk to branch-0.23 to fix MAPREDUCE-3205.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1189543 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Arun Murthy 2011-10-26 23:59:21 +00:00
parent 3ca0563e84
commit ea7c670969
16 changed files with 155 additions and 133 deletions

View File

@ -379,6 +379,10 @@ Release 0.23.0 - Unreleased
MAPREDUCE-2747. Cleaned up LinuxContainerExecutor binary sources and changed
the configuration to use yarn names. (Robert Joseph Evans via vinodkv)
MAPREDUCE-3205. Fix memory specifications to be physical rather than
virtual, allowing for a ratio between the two to be configurable. (todd
via acmurthy)
OPTIMIZATIONS
MAPREDUCE-2026. Make JobTracker.getJobCounters() and

View File

@ -296,8 +296,12 @@ public class YarnConfiguration extends Configuration {
public static final String DEFAULT_NM_REMOTE_APP_LOG_DIR = "/tmp/logs";
/** Amount of memory in GB that can be allocated for containers.*/
public static final String NM_VMEM_GB = NM_PREFIX + "resource.memory-gb";
public static final int DEFAULT_NM_VMEM_GB = 8;
public static final String NM_PMEM_MB = NM_PREFIX + "resource.memory-mb";
public static final int DEFAULT_NM_PMEM_MB = 8 * 1024;
public static final String NM_VMEM_PMEM_RATIO =
NM_PREFIX + "vmem-pmem-ratio";
public static final float DEFAULT_NM_VMEM_PMEM_RATIO = 2.1f;
/** NM Webapp address.**/
public static final String NM_WEBAPP_ADDRESS = NM_PREFIX + "webapp.address";
@ -314,10 +318,6 @@ public class YarnConfiguration extends Configuration {
public static final String NM_CONTAINER_MON_RESOURCE_CALCULATOR =
NM_PREFIX + "container-monitor.resource-calculator.class";
/** Amount of physical ram to reserve for other applications, -1 disables.*/
public static final String NM_RESERVED_MEMORY_MB =
NM_PREFIX + "reserved.memory-mb";
/** Frequency of running node health script.*/
public static final String NM_HEALTH_CHECK_INTERVAL_MS =
NM_PREFIX + "health-checker.interval-ms";

View File

@ -286,9 +286,20 @@
</property>
<property>
<description>Amount of memory in GB that can be allocated for containers.</description>
<name>yarn.nodemanager.resource.memory-gb</name>
<value>8</value>
<description>Amount of physical memory, in MB, that can be allocated
for containers.</description>
<name>yarn.nodemanager.resource.memory-mb</name>
<value>8192</value>
</property>
<property>
<description>Ratio between virtual memory to physical memory when
setting memory limits for containers. Container allocations are
expressed in terms of physical memory, and virtual memory usage
is allowed to exceed this allocation by this ratio.
</description>
<name>yarn.nodemanager.vmem-pmem-ratio</name>
<value>2.1</value>
</property>
<property>
@ -308,12 +319,6 @@
<name>yarn.nodemanager.container-monitor.resource-calculator.class</name>
</property>
<property>
<description>Amount of physical ram to reserve for other applications, -1 disables.</description>
<name>yarn.nodemanager.reserved.memory-mb</name>
<value>-1</value>
</property>
<property>
<description>Frequency of running node health script.</description>
<name>yarn.nodemanager.health-checker.interval-ms</name>

View File

@ -100,9 +100,9 @@ public synchronized void init(Configuration conf) {
this.heartBeatInterval =
conf.getLong(YarnConfiguration.NM_TO_RM_HEARTBEAT_INTERVAL_MS,
YarnConfiguration.DEFAULT_NM_TO_RM_HEARTBEAT_INTERVAL_MS);
int memory = conf.getInt(YarnConfiguration.NM_VMEM_GB, YarnConfiguration.DEFAULT_NM_VMEM_GB);
int memoryMb = conf.getInt(YarnConfiguration.NM_PMEM_MB, YarnConfiguration.DEFAULT_NM_PMEM_MB);
this.totalResource = recordFactory.newRecordInstance(Resource.class);
this.totalResource.setMemory(memory * 1024);
this.totalResource.setMemory(memoryMb);
metrics.addResource(totalResource);
super.init(conf);
}

View File

@ -275,7 +275,7 @@ public StartContainerResponse startContainer(StartContainerRequest request)
// //////////// End of parsing credentials
Container container =
new ContainerImpl(this.dispatcher, launchContext, credentials, metrics);
new ContainerImpl(getConfig(), this.dispatcher, launchContext, credentials, metrics);
ContainerId containerID = launchContext.getContainerId();
ApplicationId applicationID =
containerID.getApplicationAttemptId().getApplicationId();

View File

@ -32,6 +32,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.util.StringUtils;
@ -77,6 +78,9 @@ public class ContainerImpl implements Container {
private int exitCode = YarnConfiguration.INVALID_CONTAINER_EXIT_STATUS;
private final StringBuilder diagnostics;
/** The NM-wide configuration - not specific to this container */
private final Configuration daemonConf;
private static final Log LOG = LogFactory.getLog(Container.class);
private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
private final Map<LocalResourceRequest,String> pendingResources =
@ -90,9 +94,11 @@ public class ContainerImpl implements Container {
private final List<LocalResourceRequest> appRsrcs =
new ArrayList<LocalResourceRequest>();
public ContainerImpl(Dispatcher dispatcher,
public ContainerImpl(Configuration conf,
Dispatcher dispatcher,
ContainerLaunchContext launchContext, Credentials creds,
NodeManagerMetrics metrics) {
this.daemonConf = conf;
this.dispatcher = dispatcher;
this.launchContext = launchContext;
this.diagnostics = new StringBuilder();
@ -568,12 +574,16 @@ static class LaunchTransition extends ContainerTransition {
public void transition(ContainerImpl container, ContainerEvent event) {
// Inform the ContainersMonitor to start monitoring the container's
// resource usage.
// TODO: Fix pmem limits below
long vmemBytes =
long pmemBytes =
container.getLaunchContext().getResource().getMemory() * 1024 * 1024L;
float pmemRatio = container.daemonConf.getFloat(
YarnConfiguration.NM_VMEM_PMEM_RATIO,
YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
long vmemBytes = (long) (pmemRatio * pmemBytes);
container.dispatcher.getEventHandler().handle(
new ContainerStartMonitoringEvent(container.getContainerID(),
vmemBytes, -1));
vmemBytes, pmemBytes));
container.metrics.runningContainer();
}
}

View File

@ -28,6 +28,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
@ -39,6 +40,8 @@
import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
import com.google.inject.internal.Preconditions;
public class ContainersMonitorImpl extends AbstractService implements
ContainersMonitor {
@ -67,11 +70,6 @@ public class ContainersMonitorImpl extends AbstractService implements
*/
public static final long DISABLED_MEMORY_LIMIT = -1L;
private static final String MEMORY_USAGE_STRING =
"Memory usage of ProcessTree %s for container-id %s : Virtual %d bytes, "
+
"limit : %d bytes; Physical %d bytes, limit %d bytes";
public ContainersMonitorImpl(ContainerExecutor exec,
AsyncDispatcher dispatcher, Context context) {
super("containers-monitor");
@ -110,33 +108,33 @@ public synchronized void init(Configuration conf) {
}
}
// ///////// Virtual memory configuration //////
this.maxVmemAllottedForContainers =
conf.getLong(YarnConfiguration.NM_VMEM_GB, YarnConfiguration.DEFAULT_NM_VMEM_GB);
this.maxVmemAllottedForContainers =
this.maxVmemAllottedForContainers * 1024 * 1024 * 1024L; //Normalize
if (this.maxVmemAllottedForContainers > totalPhysicalMemoryOnNM) {
LOG.info("totalMemoryAllottedForContainers > totalPhysicalMemoryOnNM."
+ " Thrashing might happen.");
}
// ///////// Physical memory configuration //////
long reservedPmemOnNM =
conf.getLong(YarnConfiguration.NM_RESERVED_MEMORY_MB, DISABLED_MEMORY_LIMIT);
reservedPmemOnNM =
reservedPmemOnNM == DISABLED_MEMORY_LIMIT
? DISABLED_MEMORY_LIMIT
: reservedPmemOnNM * 1024 * 1024; // normalize to bytes
this.maxPmemAllottedForContainers =
conf.getLong(YarnConfiguration.NM_PMEM_MB, YarnConfiguration.DEFAULT_NM_PMEM_MB);
this.maxPmemAllottedForContainers =
this.maxPmemAllottedForContainers * 1024 * 1024L; //Normalize to bytes
if (reservedPmemOnNM == DISABLED_MEMORY_LIMIT
|| totalPhysicalMemoryOnNM == DISABLED_MEMORY_LIMIT) {
this.maxPmemAllottedForContainers = DISABLED_MEMORY_LIMIT;
} else {
this.maxPmemAllottedForContainers =
totalPhysicalMemoryOnNM - reservedPmemOnNM;
if (totalPhysicalMemoryOnNM != DISABLED_MEMORY_LIMIT &&
this.maxPmemAllottedForContainers >
totalPhysicalMemoryOnNM * 0.80f) {
LOG.warn("NodeManager configured with " +
StringUtils.humanReadableInt(maxPmemAllottedForContainers) +
" physical memory allocated to containers, which is more than " +
"80% of the total physical memory available (" +
StringUtils.humanReadableInt(totalPhysicalMemoryOnNM) +
"). Thrashing might happen.");
}
// ///////// Virtual memory configuration //////
float vmemRatio = conf.getFloat(
YarnConfiguration.NM_VMEM_PMEM_RATIO,
YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
Preconditions.checkArgument(vmemRatio > 0.99f,
YarnConfiguration.NM_VMEM_PMEM_RATIO +
" should be at least 1.0");
this.maxVmemAllottedForContainers =
(long)(vmemRatio * maxPmemAllottedForContainers);
super.init(conf);
}
@ -399,9 +397,10 @@ public void run() {
long curRssMemUsageOfAgedProcesses = pTree.getCumulativeRssmem(1);
long vmemLimit = ptInfo.getVmemLimit();
long pmemLimit = ptInfo.getPmemLimit();
LOG.info(String.format(MEMORY_USAGE_STRING, pId,
containerId.toString(), currentVmemUsage, vmemLimit,
currentPmemUsage, pmemLimit));
LOG.info(String.format(
"Memory usage of ProcessTree %s for container-id %s: ",
pId, containerId.toString()) +
formatUsageString(currentVmemUsage, vmemLimit, currentPmemUsage, pmemLimit));
boolean isMemoryOverLimit = false;
String msg = "";
@ -411,18 +410,10 @@ && isProcessTreeOverLimit(containerId.toString(),
// Container (the root process) is still alive and overflowing
// memory.
// Dump the process-tree and then clean it up.
msg =
"Container [pid="
+ pId
+ ",containerID="
+ containerId
+ "] is running beyond memory-limits. Current usage : "
+ currentVmemUsage
+ "bytes. Limit : "
+ vmemLimit
+ "bytes. Killing container. "
+ "\nDump of the process-tree for " + containerId
+ " : \n" + pTree.getProcessTreeDump();
msg = formatErrorMessage("virtual",
currentVmemUsage, vmemLimit,
currentPmemUsage, pmemLimit,
pId, containerId, pTree);
isMemoryOverLimit = true;
} else if (isPhysicalMemoryCheckEnabled()
&& isProcessTreeOverLimit(containerId.toString(),
@ -431,18 +422,10 @@ && isProcessTreeOverLimit(containerId.toString(),
// Container (the root process) is still alive and overflowing
// memory.
// Dump the process-tree and then clean it up.
msg =
"Container [pid="
+ pId
+ ",tipID="
+ containerId
+ "] is running beyond physical memory-limits."
+ " Current usage : "
+ currentPmemUsage
+ "bytes. Limit : "
+ pmemLimit
+ "bytes. Killing container. \nDump of the process-tree for "
+ containerId + " : \n" + pTree.getProcessTreeDump();
msg = formatErrorMessage("physical",
currentVmemUsage, vmemLimit,
currentPmemUsage, pmemLimit,
pId, containerId, pTree);
isMemoryOverLimit = true;
}
@ -484,6 +467,31 @@ && isProcessTreeOverLimit(containerId.toString(),
}
}
}
private String formatErrorMessage(String memTypeExceeded,
long currentVmemUsage, long vmemLimit,
long currentPmemUsage, long pmemLimit,
String pId, ContainerId containerId, ProcfsBasedProcessTree pTree) {
return
String.format("Container [pid=%s,containerID=%s] is running beyond %s memory limits. ",
pId, containerId, memTypeExceeded) +
"Current usage: " +
formatUsageString(currentVmemUsage, vmemLimit,
currentPmemUsage, pmemLimit) +
". Killing container.\n" +
"Dump of the process-tree for " + containerId + " :\n" +
pTree.getProcessTreeDump();
}
private String formatUsageString(long currentVmemUsage, long vmemLimit,
long currentPmemUsage, long pmemLimit) {
return String.format("%sb of %sb physical memory used; " +
"%sb of %sb virtual memory used",
StringUtils.humanReadableInt(currentPmemUsage),
StringUtils.humanReadableInt(pmemLimit),
StringUtils.humanReadableInt(currentVmemUsage),
StringUtils.humanReadableInt(vmemLimit));
}
}
@Override

View File

@ -30,6 +30,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.NodeHealthCheckerService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@ -83,6 +84,7 @@ public class TestNodeStatusUpdater {
int heartBeatID = 0;
volatile Error nmStartError = null;
private final List<NodeId> registeredNodes = new ArrayList<NodeId>();
private final Configuration conf = new YarnConfiguration();
@After
public void tearDown() {
@ -168,7 +170,7 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
launchContext.setContainerId(firstContainerID);
launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
launchContext.getResource().setMemory(2);
Container container = new ContainerImpl(null, launchContext, null, null);
Container container = new ContainerImpl(conf , null, launchContext, null, null);
this.context.getContainers().put(firstContainerID, container);
} else if (heartBeatID == 2) {
// Checks on the RM end
@ -192,7 +194,7 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
launchContext.setContainerId(secondContainerID);
launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
launchContext.getResource().setMemory(3);
Container container = new ContainerImpl(null, launchContext, null, null);
Container container = new ContainerImpl(conf, null, launchContext, null, null);
this.context.getContainers().put(secondContainerID, container);
} else if (heartBeatID == 3) {
// Checks on the RM end
@ -358,7 +360,7 @@ public void start() {
private YarnConfiguration createNMConfig() {
YarnConfiguration conf = new YarnConfiguration();
conf.setInt(YarnConfiguration.NM_VMEM_GB, 5); // 5GB
conf.setInt(YarnConfiguration.NM_PMEM_MB, 5*1024); // 5GB
conf.set(YarnConfiguration.NM_ADDRESS, "127.0.0.1:12345");
conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "127.0.0.1:12346");
conf.set(YarnConfiguration.NM_LOG_DIRS, new Path(basedir, "logs").toUri()

View File

@ -37,6 +37,7 @@
import java.util.Map.Entry;
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@ -45,6 +46,7 @@
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
@ -68,6 +70,7 @@
public class TestContainer {
final NodeManagerMetrics metrics = NodeManagerMetrics.create();
final Configuration conf = new YarnConfiguration();
/**
@ -384,7 +387,7 @@ private static Map<String,ByteBuffer> createServiceData(Random r) {
}
private Container newContainer(Dispatcher disp, ContainerLaunchContext ctx) {
return new ContainerImpl(disp, ctx, null, metrics);
return new ContainerImpl(conf, disp, ctx, null, metrics);
}
@SuppressWarnings("unchecked")

View File

@ -266,12 +266,13 @@ public void testContainerKillOnMemoryOverflow() throws IOException,
containerStatus.getExitStatus());
String expectedMsgPattern =
"Container \\[pid=" + pid + ",containerID=" + cId
+ "\\] is running beyond memory-limits. Current usage : "
+ "[0-9]*bytes. Limit : [0-9]*"
+ "bytes. Killing container. \nDump of the process-tree for "
+ cId + " : \n";
+ "\\] is running beyond virtual memory limits. Current usage: "
+ "[0-9.]+m?b of [0-9.]+m?b physical memory used; "
+ "[0-9.]+m?b of [0-9.]+m?b virtual memory used. "
+ "Killing container.\nDump of the process-tree for "
+ cId + " :\n";
Pattern pat = Pattern.compile(expectedMsgPattern);
Assert.assertEquals("Expected message patterns is: " + expectedMsgPattern
Assert.assertEquals("Expected message pattern is: " + expectedMsgPattern
+ "\n\nObserved message is: " + containerStatus.getDiagnostics(),
true, pat.matcher(containerStatus.getDiagnostics()).find());

View File

@ -107,7 +107,7 @@ public long getPmemAllocatedForContainers() {
launchContext.setContainerId(containerId);
launchContext.setUser(user);
Container container =
new ContainerImpl(dispatcher, launchContext, null, metrics) {
new ContainerImpl(conf, dispatcher, launchContext, null, metrics) {
@Override
public ContainerState getContainerState() {
return ContainerState.RUNNING;

View File

@ -168,7 +168,7 @@ public synchronized void start() {
getConfig().set(YarnConfiguration.NM_LOG_DIRS, logDir.getAbsolutePath());
getConfig().set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
remoteLogDir.getAbsolutePath());
getConfig().setInt(YarnConfiguration.NM_VMEM_GB, 4); // By default AM + 2 containers
getConfig().setInt(YarnConfiguration.NM_PMEM_MB, 4*1024); // By default AM + 2 containers
nodeManager = new NodeManager() {
@Override

View File

@ -241,9 +241,17 @@ Hadoop MapReduce Next Generation - Cluster Setup
*-------------------------+-------------------------+------------------------+
|| Parameter || Value || Notes |
*-------------------------+-------------------------+------------------------+
| <<<yarn.nodemanager.resource.memory-gb>>> | | |
| | Resource i.e. available memory, in GB, for given <<<NodeManager>>> | |
| | | Defines available resources on the <<<NodeManager>>>. |
| <<<yarn.nodemanager.resource.memory-mb>>> | | |
| | Resource i.e. available physical memory, in MB, for given <<<NodeManager>>> | |
| | | Defines total available resources on the <<<NodeManager>>> to be made |
| | | available to running containers |
*-------------------------+-------------------------+------------------------+
| <<<yarn.nodemanager.vmem-pmem-ratio>>> | | |
| | Maximum ratio by which virtual memory usage of tasks may exceed |
| | physical memory | |
| | | The virtual memory usage of each task may exceed its physical memory |
| | | limit by this ratio. The total amount of virtual memory used by tasks |
| | | on the NodeManager may exceed its physical memory usage by this ratio. |
*-------------------------+-------------------------+------------------------+
| <<<yarn.nodemanager.local-dirs>>> | | |
| | Comma-separated list of paths on the local filesystem where | |

View File

@ -119,8 +119,8 @@ Add the following configs to your <<<yarn-site.xml>>>
</property>
<property>
<name>yarn.nodemanager.resource.memory-gb</name>
<value>10</value>
<name>yarn.nodemanager.resource.memory-mb</name>
<value>10240</value>
<description>the amount of memory on the NodeManager in GB</description>
</property>

View File

@ -406,7 +406,8 @@ Hadoop MapReduce Next Generation - Writing YARN Applications
* Resource capability: Currently, YARN only supports memory based resource
requirements so the request should define how much memory is needed. The
value is defined in MB and has to less than the max capability of the
cluster and an exact multiple of the min capability.
cluster and an exact multiple of the min capability. Memory resources
correspond to physical memory limits imposed on the task containers.
* Priority: When asking for sets of containers, an ApplicationMaster may
define different priorities to each set. For example, the Map-Reduce
@ -774,8 +775,9 @@ Hadoop MapReduce Next Generation - Writing YARN Applications
The two things you're interested in are physical memory and virtual memory.
If you have exceeded physical memory limits your app is using too much physical
memory. If you're running a Java app, you can use -hprof to look at what is
taking up space in the heap. If you have exceeded virtual memory, things are
slightly more complicated.
taking up space in the heap. If you have exceeded virtual memory, you may
need to increase the value of the the cluster-wide configuration variable
<<<yarn.nodemanager.vmem-pmem-ratio>>>.
* Useful Links

View File

@ -1232,49 +1232,28 @@
</li>
<li>
<code>mapreduce.{map|reduce}.ulimit</code>: The slaves where
tasks are run could be configured with a ulimit value that
applies a limit to every process that is launched on the slave.
If the task, or any child that the task launches (like in
streaming), requires more than the configured limit, this option
must be used. The value is given in kilobytes. For example, to
increase the ulimit to 1G, the option should be set to 1048576.
Note that this value is a per process limit. Since it applies
to the JVM as well, the heap space given to the JVM through
the <code>mapreduce.{map|reduce}.java.opts</code> should be less
than the value configured for the ulimit. Otherwise the JVM
will not start.
</li>
<li>
<code>mapreduce.{map|reduce}.memory.mb</code>: In some
environments, administrators might have configured a total limit
on the virtual memory used by the entire process tree for a task,
including all processes launched recursively by the task or
its children, like in streaming. More details about this can be
found in the section on
<a href="ext:cluster-setup/ConfiguringMemoryParameters">
Monitoring Task Memory Usage</a> in the Cluster SetUp guide.
If a task requires more virtual memory for its entire tree,
this option
must be used. The value is given in MB. For example, to set
<code>mapreduce.{map|reduce}.memory.mb</code>:
This parameter configures how many megabytes of physical memory
the job requires for its map and reduce tasks. It must be configured
to be slightly larger than the configured Java heap size above,
to account for the fact that the JVM uses physical memory beyond
just the heap. An overhead of 20% to 30% is usually sufficient.
<br/>
Note that, the smaller the amount of memory specified for tasks
in this configuration, the larger the number of tasks that
can be run in parallel.<br/>
The value is given in MB. For example, to set
the limit to 1G, the option should be set to 1024. Note that this
value does not automatically influence the per process ulimit or
heap space. Hence, you may need to set those parameters as well
(as described above) in order to give your tasks the right amount
of memory.
<br/>
If the amount of physical memory used by your task exceeds the
configured value, the NodeManager will automatically kill the task.
</li>
<li>
<code>mapreduce.{map|reduce}.memory.physical.mb</code>:
This parameter is similar to
<code>mapreduce.{map|reduce}.memory.mb</code>, except it specifies
how much physical memory is required by a task for its entire
tree of processes. The parameter is applicable if administrators
have configured a total limit on the physical memory used by
all MapReduce tasks.
</li>
</ul>
<p>
@ -1297,7 +1276,7 @@
<p>
Note: The memory related configuration options described above
are used only for configuring the launched child tasks from the
tasktracker. Configuring the memory options for daemons is documented
NodeManager. Configuring the memory options for daemons is documented
under
<a href="ext:cluster-setup/ConfiguringEnvironmentHadoopDaemons">
Configuring the Environment of the Hadoop Daemons</a> (Cluster Setup).