|
@@ -28,6 +28,7 @@ import java.util.Map.Entry;
|
|
|
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.service.AbstractService;
|
|
|
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 class ContainersMonitorImpl extends AbstractService implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // ///////// 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
|
|
|
-
|
|
|
- if (reservedPmemOnNM == DISABLED_MEMORY_LIMIT
|
|
|
- || totalPhysicalMemoryOnNM == DISABLED_MEMORY_LIMIT) {
|
|
|
- this.maxPmemAllottedForContainers = DISABLED_MEMORY_LIMIT;
|
|
|
- } else {
|
|
|
- this.maxPmemAllottedForContainers =
|
|
|
- totalPhysicalMemoryOnNM - reservedPmemOnNM;
|
|
|
+ this.maxPmemAllottedForContainers =
|
|
|
+ conf.getLong(YarnConfiguration.NM_PMEM_MB, YarnConfiguration.DEFAULT_NM_PMEM_MB);
|
|
|
+ this.maxPmemAllottedForContainers =
|
|
|
+ this.maxPmemAllottedForContainers * 1024 * 1024L; //Normalize to bytes
|
|
|
+
|
|
|
+ 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 class ContainersMonitorImpl extends AbstractService implements
|
|
|
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 @@ public class ContainersMonitorImpl extends AbstractService implements
|
|
|
// 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 @@ public class ContainersMonitorImpl extends AbstractService implements
|
|
|
// 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 @@ public class ContainersMonitorImpl extends AbstractService implements
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ 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
|