YARN-10471. Prevent logs for any container from becoming larger than a configurable size. Contributed by Eric Payne

(cherry picked from commit 8ee6bc2518)
This commit is contained in:
Jim Brennan 2020-10-29 19:49:18 +00:00
parent 91a3d298b9
commit cdaef111d5
6 changed files with 347 additions and 4 deletions

View File

@ -78,4 +78,9 @@ public class ContainerExitStatus {
*/
public static final int KILLED_BY_CONTAINER_SCHEDULER = -108;
/**
* Container was terminated for generating excess log data.
*/
public static final int KILLED_FOR_EXCESS_LOGS = -109;
}

View File

@ -1913,6 +1913,25 @@ public static boolean isAclEnabled(Configuration conf) {
public static final String APPLICATION_TAG_BASED_PLACEMENT_USER_WHITELIST =
APPLICATION_TAG_BASED_PLACEMENT_PREFIX + ".username.whitelist";
/** Enable switch for container log monitoring. */
public static final String NM_CONTAINER_LOG_MONITOR_ENABLED =
NM_PREFIX + "container-log-monitor.enable";
public static final boolean DEFAULT_NM_CONTAINER_LOG_MONITOR_ENABLED = false;
/** How often to monitor logs generated by containers. */
public static final String NM_CONTAINER_LOG_MON_INTERVAL_MS =
NM_PREFIX + "container-log-monitor.interval-ms";
public static final int DEFAULT_NM_CONTAINER_LOG_MON_INTERVAL_MS = 60000;
/** The disk space limit for a single container log directory. */
public static final String NM_CONTAINER_LOG_DIR_SIZE_LIMIT_BYTES =
NM_PREFIX + "container-log-monitor.dir-size-limit-bytes";
public static final long DEFAULT_NM_CONTAINER_LOG_DIR_SIZE_LIMIT_BYTES =
1000000000L;
/** The disk space limit for all of a container's logs. */
public static final String NM_CONTAINER_LOG_TOTAL_SIZE_LIMIT_BYTES =
NM_PREFIX + "container-log-monitor.total-size-limit-bytes";
public static final long DEFAULT_NM_CONTAINER_LOG_TOTAL_SIZE_LIMIT_BYTES =
10000000000L;
/** Enable/disable container metrics. */
@Private
public static final String NM_CONTAINER_METRICS_ENABLE =

View File

@ -1659,6 +1659,34 @@
<name>yarn.nodemanager.container-monitor.interval-ms</name>
</property>
<property>
<description>Flag to enable the container log monitor which enforces
container log directory size limits.</description>
<name>yarn.nodemanager.container-log-monitor.enable</name>
<value>false</value>
</property>
<property>
<description>How often to check the usage of a container's log directories
in milliseconds</description>
<name>yarn.nodemanager.container-log-monitor.interval-ms</name>
<value>60000</value>
</property>
<property>
<description>The disk space limit, in bytes, for a single
container log directory</description>
<name>yarn.nodemanager.container-log-monitor.dir-size-limit-bytes</name>
<value>1000000000</value>
</property>
<property>
<description>The disk space limit, in bytes, for all of a container's
logs</description>
<name>yarn.nodemanager.container-log-monitor.total-size-limit-bytes</name>
<value>10000000000</value>
</property>
<property>
<description>Class that calculates containers current resource utilization.
If not set, the value for yarn.nodemanager.resource-calculator.class will

View File

@ -28,6 +28,7 @@
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
@ -45,11 +46,14 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
import org.apache.hadoop.yarn.server.nodemanager.webapp.ContainerLogsUtils;
import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
import java.util.Arrays;
import java.io.File;
import java.util.Map;
import java.util.List;
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap;
@ -67,6 +71,10 @@ public class ContainersMonitorImpl extends AbstractService implements
private long monitoringInterval;
private MonitoringThread monitoringThread;
private int logCheckInterval;
private LogMonitorThread logMonitorThread;
private long logDirSizeLimit;
private long logTotalSizeLimit;
private CGroupElasticMemoryController oomListenerThread;
private boolean containerMetricsEnabled;
private long containerMetricsPeriodMs;
@ -94,6 +102,7 @@ public class ContainersMonitorImpl extends AbstractService implements
private boolean elasticMemoryEnforcement;
private boolean strictMemoryEnforcement;
private boolean containersMonitorEnabled;
private boolean logMonitorEnabled;
private long maxVCoresAllottedForContainers;
@ -122,6 +131,8 @@ public ContainersMonitorImpl(ContainerExecutor exec,
this.monitoringThread = new MonitoringThread();
this.logMonitorThread = new LogMonitorThread();
this.containersUtilization = ResourceUtilization.newInstance(0, 0, 0.0f);
}
@ -133,6 +144,16 @@ protected void serviceInit(Configuration myConf) throws Exception {
this.conf.getLong(YarnConfiguration.NM_RESOURCE_MON_INTERVAL_MS,
YarnConfiguration.DEFAULT_NM_RESOURCE_MON_INTERVAL_MS));
this.logCheckInterval =
conf.getInt(YarnConfiguration.NM_CONTAINER_LOG_MON_INTERVAL_MS,
YarnConfiguration.DEFAULT_NM_CONTAINER_LOG_MON_INTERVAL_MS);
this.logDirSizeLimit =
conf.getLong(YarnConfiguration.NM_CONTAINER_LOG_DIR_SIZE_LIMIT_BYTES,
YarnConfiguration.DEFAULT_NM_CONTAINER_LOG_DIR_SIZE_LIMIT_BYTES);
this.logTotalSizeLimit =
conf.getLong(YarnConfiguration.NM_CONTAINER_LOG_TOTAL_SIZE_LIMIT_BYTES,
YarnConfiguration.DEFAULT_NM_CONTAINER_LOG_TOTAL_SIZE_LIMIT_BYTES);
this.resourceCalculatorPlugin =
ResourceCalculatorPlugin.getContainersMonitorPlugin(this.conf);
LOG.info("Using ResourceCalculatorPlugin: {}",
@ -214,6 +235,11 @@ protected void serviceInit(Configuration myConf) throws Exception {
isContainerMonitorEnabled() && monitoringInterval > 0;
LOG.info("ContainersMonitor enabled: {}", containersMonitorEnabled);
logMonitorEnabled =
conf.getBoolean(YarnConfiguration.NM_CONTAINER_LOG_MONITOR_ENABLED,
YarnConfiguration.DEFAULT_NM_CONTAINER_LOG_MONITOR_ENABLED);
LOG.info("Container Log Monitor Enabled: "+ logMonitorEnabled);
nodeCpuPercentageForYARN =
NodeManagerHardwareUtils.getNodeCpuPercentage(this.conf);
@ -284,13 +310,16 @@ protected void serviceStart() throws Exception {
if (oomListenerThread != null) {
oomListenerThread.start();
}
if (logMonitorEnabled) {
this.logMonitorThread.start();
}
super.serviceStart();
}
@Override
protected void serviceStop() throws Exception {
stopped = true;
if (containersMonitorEnabled) {
stopped = true;
this.monitoringThread.interrupt();
try {
this.monitoringThread.join();
@ -306,6 +335,13 @@ protected void serviceStop() throws Exception {
}
}
}
if (logMonitorEnabled) {
this.logMonitorThread.interrupt();
try {
this.logMonitorThread.join();
} catch (InterruptedException e) {
}
}
super.serviceStop();
}
@ -752,7 +788,8 @@ && isProcessTreeOverLimit(containerId.toString(),
containerExitStatus = ContainerExitStatus.KILLED_EXCEEDED_PMEM;
}
if (isMemoryOverLimit) {
if (isMemoryOverLimit
&& trackingContainers.remove(containerId) != null) {
// Virtual or physical memory over limit. Fail the container and
// remove
// the corresponding process tree
@ -766,7 +803,6 @@ && isProcessTreeOverLimit(containerId.toString(),
eventDispatcher.getEventHandler().handle(
new ContainerKillEvent(containerId,
containerExitStatus, msg));
trackingContainers.remove(containerId);
LOG.info("Removed ProcessTree with root {}", pId);
}
}
@ -834,6 +870,72 @@ private String formatUsageString(long currentVmemUsage, long vmemLimit,
}
}
private class LogMonitorThread extends Thread {
LogMonitorThread() {
super("Container Log Monitor");
}
@Override
public void run() {
while (!stopped && !Thread.currentThread().isInterrupted()) {
for (Entry<ContainerId, ProcessTreeInfo> entry :
trackingContainers.entrySet()) {
ContainerId containerId = entry.getKey();
ProcessTreeInfo ptInfo = entry.getValue();
Container container = context.getContainers().get(containerId);
if (container == null) {
continue;
}
try {
List<File> logDirs = ContainerLogsUtils.getContainerLogDirs(
containerId, container.getUser(), context);
long totalLogDataBytes = 0;
for (File dir : logDirs) {
long currentDirSizeBytes = FileUtil.getDU(dir);
totalLogDataBytes += currentDirSizeBytes;
String killMsg = null;
if (currentDirSizeBytes > logDirSizeLimit) {
killMsg = String.format(
"Container [pid=%s,containerID=%s] is logging beyond "
+ "the container single log directory limit.%n"
+ "Limit: %d Log Directory Size: %d Log Directory: %s"
+ "%nKilling container.%n",
ptInfo.getPID(), containerId, logDirSizeLimit,
currentDirSizeBytes, dir);
} else if (totalLogDataBytes > logTotalSizeLimit) {
killMsg = String.format(
"Container [pid=%s,containerID=%s] is logging beyond "
+ "the container total log limit.%n"
+ "Limit: %d Total Size: >=%d"
+ "%nKilling container.%n",
ptInfo.getPID(), containerId, logTotalSizeLimit,
totalLogDataBytes);
}
if (killMsg != null
&& trackingContainers.remove(containerId) != null) {
LOG.warn(killMsg);
eventDispatcher.getEventHandler().handle(
new ContainerKillEvent(containerId,
ContainerExitStatus.KILLED_FOR_EXCESS_LOGS, killMsg));
LOG.info("Removed ProcessTree with root " + ptInfo.getPID());
break;
}
}
} catch (Exception e) {
LOG.warn("Uncaught exception in ContainerMemoryManager "
+ "while monitoring log usage for " + containerId, e);
}
}
try {
Thread.sleep(logCheckInterval);
} catch (InterruptedException e) {
LOG.info("Log monitor thread was interrupted. "
+ "Stopping container log monitoring.");
}
}
}
}
private void updateContainerMetrics(ContainersMonitorEvent monitoringEvent) {
if (!containerMetricsEnabled || monitoringEvent == null) {
return;

View File

@ -21,9 +21,13 @@
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.timeout;
import static org.mockito.Mockito.verify;
import java.io.BufferedReader;
import java.io.File;
@ -32,12 +36,14 @@
import java.io.PrintWriter;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
import java.util.function.Supplier;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
@ -61,12 +67,19 @@
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
import org.apache.hadoop.yarn.server.nodemanager.Context;
import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
@ -76,6 +89,7 @@
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import org.slf4j.LoggerFactory;
@ -88,6 +102,7 @@ public TestContainersMonitor() throws UnsupportedFileSystemException {
static {
LOG = LoggerFactory.getLogger(TestContainersMonitor.class);
}
@Before
public void setup() throws IOException {
conf.setClass(
@ -353,6 +368,164 @@ public void testContainerKillOnMemoryOverflow() throws IOException,
.build()));
}
@SuppressWarnings("unchecked")
@Test
public void testContainerKillOnExcessLogDirectory() throws Exception {
final String user = "someuser";
ApplicationId appId = ApplicationId.newInstance(1, 1);
ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
ContainerId cid = ContainerId.newContainerId(attemptId, 1);
Application app = mock(Application.class);
doReturn(user).when(app).getUser();
doReturn(appId).when(app).getAppId();
Container container = mock(Container.class);
doReturn(cid).when(container).getContainerId();
doReturn(user).when(container).getUser();
File containerLogDir = new File(new File(localLogDir, appId.toString()),
cid.toString());
containerLogDir.mkdirs();
LocalDirsHandlerService mockDirsHandler =
mock(LocalDirsHandlerService.class);
doReturn(Collections.singletonList(localLogDir.getAbsolutePath()))
.when(mockDirsHandler).getLogDirsForRead();
Context ctx = new NMContext(context.getContainerTokenSecretManager(),
context.getNMTokenSecretManager(), mockDirsHandler,
context.getApplicationACLsManager(), context.getNMStateStore(),
false, conf);
Configuration monitorConf = new Configuration(conf);
monitorConf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
monitorConf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false);
monitorConf.setBoolean(YarnConfiguration.NM_CONTAINER_METRICS_ENABLE,
false);
monitorConf.setBoolean(YarnConfiguration.NM_CONTAINER_LOG_MONITOR_ENABLED,
true);
monitorConf.setLong(
YarnConfiguration.NM_CONTAINER_LOG_DIR_SIZE_LIMIT_BYTES, 10);
monitorConf.setLong(
YarnConfiguration.NM_CONTAINER_LOG_TOTAL_SIZE_LIMIT_BYTES, 10000000);
monitorConf.setLong(YarnConfiguration.NM_CONTAINER_LOG_MON_INTERVAL_MS,
10);
EventHandler mockHandler = mock(EventHandler.class);
AsyncDispatcher mockDispatcher = mock(AsyncDispatcher.class);
doReturn(mockHandler).when(mockDispatcher).getEventHandler();
ContainersMonitor monitor = new ContainersMonitorImpl(
mock(ContainerExecutor.class), mockDispatcher, ctx);
monitor.init(monitorConf);
monitor.start();
Event event;
try {
ctx.getApplications().put(appId, app);
ctx.getContainers().put(cid, container);
monitor.handle(new ContainerStartMonitoringEvent(cid, 1, 1, 1, 0, 0));
PrintWriter fileWriter = new PrintWriter(new File(containerLogDir,
"log"));
fileWriter.write("This container is logging too much.");
fileWriter.close();
ArgumentCaptor<Event> captor = ArgumentCaptor.forClass(Event.class);
verify(mockHandler, timeout(10000)).handle(captor.capture());
event = captor.getValue();
} finally {
monitor.stop();
}
assertTrue("Expected a kill event", event instanceof ContainerKillEvent);
ContainerKillEvent cke = (ContainerKillEvent) event;
assertEquals("Unexpected container exit status",
ContainerExitStatus.KILLED_FOR_EXCESS_LOGS,
cke.getContainerExitStatus());
}
@SuppressWarnings("unchecked")
@Test
public void testContainerKillOnExcessTotalLogs() throws Exception {
final String user = "someuser";
ApplicationId appId = ApplicationId.newInstance(1, 1);
ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
ContainerId cid = ContainerId.newContainerId(attemptId, 1);
Application app = mock(Application.class);
doReturn(user).when(app).getUser();
doReturn(appId).when(app).getAppId();
Container container = mock(Container.class);
doReturn(cid).when(container).getContainerId();
doReturn(user).when(container).getUser();
File logDir1 = new File(localLogDir, "dir1");
File logDir2 = new File(localLogDir, "dir2");
List<String> logDirs = new ArrayList<>();
logDirs.add(logDir1.getAbsolutePath());
logDirs.add(logDir2.getAbsolutePath());
LocalDirsHandlerService mockDirsHandler =
mock(LocalDirsHandlerService.class);
doReturn(logDirs).when(mockDirsHandler).getLogDirsForRead();
Context ctx = new NMContext(context.getContainerTokenSecretManager(),
context.getNMTokenSecretManager(), mockDirsHandler,
context.getApplicationACLsManager(), context.getNMStateStore(),
false, conf);
File clogDir1 = new File(new File(logDir1, appId.toString()),
cid.toString());
clogDir1.mkdirs();
File clogDir2 = new File(new File(logDir2, appId.toString()),
cid.toString());
clogDir2.mkdirs();
Configuration monitorConf = new Configuration(conf);
monitorConf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
monitorConf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false);
monitorConf.setBoolean(YarnConfiguration.NM_CONTAINER_METRICS_ENABLE,
false);
monitorConf.setBoolean(YarnConfiguration.NM_CONTAINER_LOG_MONITOR_ENABLED,
true);
monitorConf.setLong(
YarnConfiguration.NM_CONTAINER_LOG_DIR_SIZE_LIMIT_BYTES, 100000);
monitorConf.setLong(
YarnConfiguration.NM_CONTAINER_LOG_TOTAL_SIZE_LIMIT_BYTES, 15);
monitorConf.setLong(YarnConfiguration.NM_CONTAINER_LOG_MON_INTERVAL_MS,
10);
monitorConf.set(YarnConfiguration.NM_LOG_DIRS, logDir1.getAbsolutePath()
+ "," + logDir2.getAbsolutePath());
EventHandler mockHandler = mock(EventHandler.class);
AsyncDispatcher mockDispatcher = mock(AsyncDispatcher.class);
doReturn(mockHandler).when(mockDispatcher).getEventHandler();
ContainersMonitor monitor = new ContainersMonitorImpl(
mock(ContainerExecutor.class), mockDispatcher, ctx);
monitor.init(monitorConf);
monitor.start();
Event event;
try {
ctx.getApplications().put(appId, app);
ctx.getContainers().put(cid, container);
monitor.handle(new ContainerStartMonitoringEvent(cid, 1, 1, 1, 0, 0));
PrintWriter fileWriter = new PrintWriter(new File(clogDir1, "log"));
fileWriter.write("0123456789");
fileWriter.close();
Thread.sleep(1000);
verify(mockHandler, never()).handle(any(Event.class));
fileWriter = new PrintWriter(new File(clogDir2, "log"));
fileWriter.write("0123456789");
fileWriter.close();
ArgumentCaptor<Event> captor = ArgumentCaptor.forClass(Event.class);
verify(mockHandler, timeout(10000)).handle(captor.capture());
event = captor.getValue();
} finally {
monitor.stop();
}
assertTrue("Expected a kill event", event instanceof ContainerKillEvent);
ContainerKillEvent cke = (ContainerKillEvent) event;
assertEquals("Unexpected container exit status",
ContainerExitStatus.KILLED_FOR_EXCESS_LOGS,
cke.getContainerExitStatus());
}
@Test(timeout = 20000)
public void testContainerMonitorMemFlags() {
ContainersMonitor cm = null;

View File

@ -203,3 +203,19 @@ The following settings need to be set in *yarn-site.xml*.
<name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
<value>org.apache.hadoop.mapred.ShuffleHandler</value>
</property>
Prevent Container Logs From Getting Too Big
-------------------------------------------
This allows a cluster admin to configure a cluster such that a task attempt will be killed if any container log exceeds a configured size. This helps prevent logs from filling disks and also prevent the need to aggregate enormous logs.
### Configuration
The following parameters can be used to configure the container log dir sizes.
| Configuration Name | Allowed Values | Description |
|:---- |:---- |:---- |
| `yarn.nodemanager.container-log-monitor.enable` | true, false | Flag to enable the container log monitor which enforces container log directory size limits. Default is false. |
| `yarn.nodemanager.container-log-monitor.interval-ms` | Positive integer | How often to check the usage of a container's log directories in milliseconds. Default is 60000 ms. |
| `yarn.nodemanager.container-log-monitor.dir-size-limit-bytes` | Long | The disk space limit, in bytes, for a single container log directory. Default is 1000000000. |
| `yarn.nodemanager.container-log-monitor.total-size-limit-bytes` | Long | The disk space limit, in bytes, for all of a container's logs. The default is 10000000000. |