|
@@ -6,9 +6,9 @@
|
|
|
* to you under the Apache License, Version 2.0 (the
|
|
|
* "License"); you may not use this file except in compliance
|
|
|
* with the License. You may obtain a copy of the License at
|
|
|
- * <p>
|
|
|
+ *
|
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
|
|
- * <p>
|
|
|
+ *
|
|
|
* Unless required by applicable law or agreed to in writing, software
|
|
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
@@ -18,10 +18,12 @@
|
|
|
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
+import org.apache.commons.lang.builder.HashCodeBuilder;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
+import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
@@ -30,7 +32,7 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.Comparator;
|
|
|
+import java.util.Collections;
|
|
|
|
|
|
import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler.CGROUP_FILE_TASKS;
|
|
|
import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler.CGROUP_PARAM_MEMORY_MEMSW_USAGE_BYTES;
|
|
@@ -46,66 +48,60 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
|
|
|
public class DefaultOOMHandler implements Runnable {
|
|
|
protected static final Log LOG = LogFactory
|
|
|
.getLog(DefaultOOMHandler.class);
|
|
|
- private Context context;
|
|
|
- private boolean virtual;
|
|
|
- private CGroupsHandler cgroups;
|
|
|
+ private final Context context;
|
|
|
+ private final String memoryStatFile;
|
|
|
+ private final CGroupsHandler cgroups;
|
|
|
|
|
|
/**
|
|
|
* Create an OOM handler.
|
|
|
* This has to be public to be able to construct through reflection.
|
|
|
* @param context node manager context to work with
|
|
|
- * @param testVirtual Test virtual memory or physical
|
|
|
+ * @param enforceVirtualMemory true if virtual memory needs to be checked,
|
|
|
+ * false if physical memory needs to be checked instead
|
|
|
*/
|
|
|
- public DefaultOOMHandler(Context context, boolean testVirtual) {
|
|
|
+ public DefaultOOMHandler(Context context, boolean enforceVirtualMemory) {
|
|
|
this.context = context;
|
|
|
- this.virtual = testVirtual;
|
|
|
- this.cgroups = ResourceHandlerModule.getCGroupsHandler();
|
|
|
+ this.memoryStatFile = enforceVirtualMemory ?
|
|
|
+ CGROUP_PARAM_MEMORY_MEMSW_USAGE_BYTES :
|
|
|
+ CGROUP_PARAM_MEMORY_USAGE_BYTES;
|
|
|
+ this.cgroups = getCGroupsHandler();
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- void setCGroupsHandler(CGroupsHandler handler) {
|
|
|
- cgroups = handler;
|
|
|
+ protected CGroupsHandler getCGroupsHandler() {
|
|
|
+ return ResourceHandlerModule.getCGroupsHandler();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Kill the container, if it has exceeded its request.
|
|
|
- *
|
|
|
- * @param container Container to check
|
|
|
- * @param fileName CGroup filename (physical or swap/virtual)
|
|
|
- * @return true, if the container was preempted
|
|
|
+ * Check if a given container exceeds its limits.
|
|
|
*/
|
|
|
- private boolean killContainerIfOOM(Container container, String fileName) {
|
|
|
+ private boolean isContainerOutOfLimit(Container container) {
|
|
|
+ boolean outOfLimit = false;
|
|
|
+
|
|
|
String value = null;
|
|
|
try {
|
|
|
value = cgroups.getCGroupParam(CGroupsHandler.CGroupController.MEMORY,
|
|
|
- container.getContainerId().toString(),
|
|
|
- fileName);
|
|
|
+ container.getContainerId().toString(), memoryStatFile);
|
|
|
long usage = Long.parseLong(value);
|
|
|
long request = container.getResource().getMemorySize() * 1024 * 1024;
|
|
|
|
|
|
// Check if the container has exceeded its limits.
|
|
|
if (usage > request) {
|
|
|
- // Kill the container
|
|
|
- // We could call the regular cleanup but that sends a
|
|
|
- // SIGTERM first that cannot be handled by frozen processes.
|
|
|
- // Walk through the cgroup
|
|
|
- // tasks file and kill all processes in it
|
|
|
- sigKill(container);
|
|
|
+ outOfLimit = true;
|
|
|
String message = String.format(
|
|
|
- "Container %s was killed by elastic cgroups OOM handler using %d " +
|
|
|
+ "Container %s is out of its limits, using %d " +
|
|
|
"when requested only %d",
|
|
|
container.getContainerId(), usage, request);
|
|
|
LOG.warn(message);
|
|
|
- return true;
|
|
|
}
|
|
|
} catch (ResourceHandlerException ex) {
|
|
|
LOG.warn(String.format("Could not access memory resource for %s",
|
|
|
container.getContainerId()), ex);
|
|
|
} catch (NumberFormatException ex) {
|
|
|
- LOG.warn(String.format("Could not parse %s in %s",
|
|
|
- value, container.getContainerId()));
|
|
|
+ LOG.warn(String.format("Could not parse %s in %s", value,
|
|
|
+ container.getContainerId()));
|
|
|
}
|
|
|
- return false;
|
|
|
+ return outOfLimit;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -168,21 +164,16 @@ public class DefaultOOMHandler implements Runnable {
|
|
|
/**
|
|
|
* It is called when the node is under an OOM condition. All processes in
|
|
|
* all sub-cgroups are suspended. We need to act fast, so that we do not
|
|
|
- * affect the overall system utilization.
|
|
|
- * In general we try to find a newly run container that exceeded its limits.
|
|
|
- * The justification is cost, since probably this is the one that has
|
|
|
- * accumulated the least amount of uncommitted data so far.
|
|
|
- * We continue the process until the OOM is resolved.
|
|
|
+ * affect the overall system utilization. In general we try to find a
|
|
|
+ * newly launched container that exceeded its limits. The justification is
|
|
|
+ * cost, since probably this is the one that has accumulated the least
|
|
|
+ * amount of uncommitted data so far. OPPORTUNISTIC containers are always
|
|
|
+ * killed before any GUARANTEED containers are considered. We continue the
|
|
|
+ * process until the OOM is resolved.
|
|
|
*/
|
|
|
@Override
|
|
|
public void run() {
|
|
|
try {
|
|
|
- // Reverse order by start time
|
|
|
- Comparator<Container> comparator = (Container o1, Container o2) -> {
|
|
|
- long order = o1.getContainerStartTime() - o2.getContainerStartTime();
|
|
|
- return order > 0 ? -1 : order < 0 ? 1 : 0;
|
|
|
- };
|
|
|
-
|
|
|
// We kill containers until the kernel reports the OOM situation resolved
|
|
|
// Note: If the kernel has a delay this may kill more than necessary
|
|
|
while (true) {
|
|
@@ -194,61 +185,135 @@ public class DefaultOOMHandler implements Runnable {
|
|
|
break;
|
|
|
}
|
|
|
|
|
|
- // The first pass kills a recent container
|
|
|
- // that uses more than its request
|
|
|
- ArrayList<Container> containers = new ArrayList<>();
|
|
|
- containers.addAll(context.getContainers().values());
|
|
|
- // Note: Sorting may take a long time with 10K+ containers
|
|
|
- // but it is acceptable now with low number of containers per node
|
|
|
- containers.sort(comparator);
|
|
|
-
|
|
|
- // Kill the latest container that exceeded its request
|
|
|
- boolean found = false;
|
|
|
- for (Container container : containers) {
|
|
|
- if (!virtual) {
|
|
|
- if (killContainerIfOOM(container,
|
|
|
- CGROUP_PARAM_MEMORY_USAGE_BYTES)) {
|
|
|
- found = true;
|
|
|
- break;
|
|
|
- }
|
|
|
- } else {
|
|
|
- if (killContainerIfOOM(container,
|
|
|
- CGROUP_PARAM_MEMORY_MEMSW_USAGE_BYTES)) {
|
|
|
- found = true;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- if (found) {
|
|
|
- continue;
|
|
|
- }
|
|
|
+ boolean containerKilled = killContainer();
|
|
|
|
|
|
- // We have not found any containers that ran out of their limit,
|
|
|
- // so we will kill the latest one. This can happen, if all use
|
|
|
- // close to their request and one of them requests a big block
|
|
|
- // triggering the OOM freeze.
|
|
|
- // Currently there is no other way to identify the outstanding one.
|
|
|
- if (containers.size() > 0) {
|
|
|
- Container container = containers.get(0);
|
|
|
- sigKill(container);
|
|
|
- String message = String.format(
|
|
|
- "Newest container %s killed by elastic cgroups OOM handler using",
|
|
|
- container.getContainerId());
|
|
|
- LOG.warn(message);
|
|
|
- continue;
|
|
|
+ if (!containerKilled) {
|
|
|
+ // This can happen, if SIGKILL did not clean up
|
|
|
+ // non-PGID or containers or containers launched by other users
|
|
|
+ // or if a process was put to the root YARN cgroup.
|
|
|
+ throw new YarnRuntimeException(
|
|
|
+ "Could not find any containers but CGroups " +
|
|
|
+ "reserved for containers ran out of memory. " +
|
|
|
+ "I am giving up");
|
|
|
}
|
|
|
-
|
|
|
- // This can happen, if SIGKILL did not clean up
|
|
|
- // non-PGID or containers or containers launched by other users
|
|
|
- // or if a process was put to the root YARN cgroup.
|
|
|
- throw new YarnRuntimeException(
|
|
|
- "Could not find any containers but CGroups " +
|
|
|
- "reserved for containers ran out of memory. " +
|
|
|
- "I am giving up");
|
|
|
}
|
|
|
} catch (ResourceHandlerException ex) {
|
|
|
- LOG.warn("Could not fecth OOM status. " +
|
|
|
+ LOG.warn("Could not fetch OOM status. " +
|
|
|
"This is expected at shutdown. Exiting.", ex);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Choose and kill a container in case of OOM. We try to find the most
|
|
|
+ * recently launched OPPORTUNISTIC container that exceeds its limit
|
|
|
+ * and fall back to the most recently launched OPPORTUNISTIC container
|
|
|
+ * If there is no such container found, we choose to kill a GUARANTEED
|
|
|
+ * container in the same way.
|
|
|
+ * @return true if a container is killed, false otherwise
|
|
|
+ */
|
|
|
+ protected boolean killContainer() {
|
|
|
+ boolean containerKilled = false;
|
|
|
+
|
|
|
+ ArrayList<ContainerCandidate> candidates = new ArrayList<>(0);
|
|
|
+ for (Container container : context.getContainers().values()) {
|
|
|
+ candidates.add(
|
|
|
+ new ContainerCandidate(container, isContainerOutOfLimit(container)));
|
|
|
+ }
|
|
|
+ Collections.sort(candidates);
|
|
|
+
|
|
|
+ if (candidates.size() > 0) {
|
|
|
+ ContainerCandidate candidate = candidates.get(0);
|
|
|
+ sigKill(candidate.container);
|
|
|
+ String message = String.format(
|
|
|
+ "container %s killed by elastic cgroups OOM handler.",
|
|
|
+ candidate.container.getContainerId());
|
|
|
+ LOG.warn(message);
|
|
|
+ containerKilled = true;
|
|
|
+ }
|
|
|
+ return containerKilled;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Note: this class has a natural ordering that is inconsistent with equals.
|
|
|
+ */
|
|
|
+ private static class ContainerCandidate
|
|
|
+ implements Comparable<ContainerCandidate> {
|
|
|
+ private final boolean outOfLimit;
|
|
|
+ final Container container;
|
|
|
+
|
|
|
+ ContainerCandidate(Container container, boolean outOfLimit) {
|
|
|
+ this.outOfLimit = outOfLimit;
|
|
|
+ this.container = container;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Order two containers by their execution type, followed by
|
|
|
+ * their out-of-limit status and then launch time. Opportunistic
|
|
|
+ * containers are ordered before Guaranteed containers. If two
|
|
|
+ * containers are of the same execution type, the one that is
|
|
|
+ * out of its limits is ordered before the one that isn't. If
|
|
|
+ * two containers have the same execution type and out-of-limit
|
|
|
+ * status, the one that's launched later is ordered before the
|
|
|
+ * other one.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public int compareTo(ContainerCandidate o) {
|
|
|
+ boolean isThisOpportunistic = isOpportunistic(container);
|
|
|
+ boolean isOtherOpportunistic = isOpportunistic(o.container);
|
|
|
+ int ret = Boolean.compare(isOtherOpportunistic, isThisOpportunistic);
|
|
|
+ if (ret == 0) {
|
|
|
+ // the two containers are of the same execution type, order them
|
|
|
+ // by their out-of-limit status.
|
|
|
+ int outOfLimitRet = Boolean.compare(o.outOfLimit, outOfLimit);
|
|
|
+ if (outOfLimitRet == 0) {
|
|
|
+ // the two containers are also of the same out-of-limit status,
|
|
|
+ // order them by their launch time
|
|
|
+ ret = Long.compare(o.container.getContainerLaunchTime(),
|
|
|
+ this.container.getContainerLaunchTime());
|
|
|
+ } else {
|
|
|
+ ret = outOfLimitRet;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return ret;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean equals(Object obj) {
|
|
|
+ if (this == obj) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ if (obj == null) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ if (this.getClass() != obj.getClass()) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ ContainerCandidate other = (ContainerCandidate) obj;
|
|
|
+ if (this.outOfLimit != other.outOfLimit) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ if (this.container == null) {
|
|
|
+ return other.container == null;
|
|
|
+ } else {
|
|
|
+ return this.container.equals(other.container);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int hashCode() {
|
|
|
+ return new HashCodeBuilder().append(container).append(outOfLimit)
|
|
|
+ .toHashCode();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Check if a container is OPPORTUNISTIC or not. A container is
|
|
|
+ * considered OPPORTUNISTIC only if its execution type is not
|
|
|
+ * null and is OPPORTUNISTIC.
|
|
|
+ */
|
|
|
+ private static boolean isOpportunistic(Container container) {
|
|
|
+ return container.getContainerTokenIdentifier() != null &&
|
|
|
+ ExecutionType.OPPORTUNISTIC.equals(
|
|
|
+ container.getContainerTokenIdentifier().getExecutionType());
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|