|
@@ -37,6 +37,7 @@ import java.io.ByteArrayOutputStream;
|
|
|
import java.io.DataInputStream;
|
|
|
import java.io.EOFException;
|
|
|
import java.io.File;
|
|
|
+import java.io.FileNotFoundException;
|
|
|
import java.io.FileWriter;
|
|
|
import java.io.IOException;
|
|
|
import java.io.PrintStream;
|
|
@@ -50,14 +51,18 @@ import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.ConcurrentMap;
|
|
|
|
|
|
import org.junit.Assert;
|
|
|
-
|
|
|
import org.apache.commons.lang.StringUtils;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.FileContext;
|
|
|
+import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.RemoteIterator;
|
|
|
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
@@ -73,6 +78,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResource;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
|
|
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.URL;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
@@ -85,29 +91,32 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat;
|
|
|
+import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
|
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
|
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader;
|
|
|
import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
|
|
|
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.TestContainerManager;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppStartedEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerContainerFinishedEvent;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
+import org.apache.hadoop.yarn.util.Records;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.ArgumentCaptor;
|
|
|
import org.mockito.Mockito;
|
|
|
import org.mortbay.util.MultiException;
|
|
|
|
|
|
-
|
|
|
-
|
|
|
//@Ignore
|
|
|
public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
|
|
@@ -178,7 +187,8 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
BuilderUtils.newApplicationAttemptId(application1, 1);
|
|
|
ContainerId container11 = BuilderUtils.newContainerId(appAttemptId, 1);
|
|
|
// Simulate log-file creation
|
|
|
- writeContainerLogs(app1LogDir, container11);
|
|
|
+ writeContainerLogs(app1LogDir, container11, new String[] { "stdout",
|
|
|
+ "stderr", "syslog" });
|
|
|
logAggregationService.handle(
|
|
|
new LogHandlerContainerFinishedEvent(container11, 0));
|
|
|
|
|
@@ -206,6 +216,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
Path logFilePath =
|
|
|
logAggregationService.getRemoteNodeLogFileForApp(application1,
|
|
|
this.user);
|
|
|
+
|
|
|
Assert.assertTrue("Log file [" + logFilePath + "] not found", new File(
|
|
|
logFilePath.toUri().getPath()).exists());
|
|
|
|
|
@@ -261,7 +272,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
Assert.assertFalse(new File(logAggregationService
|
|
|
.getRemoteNodeLogFileForApp(application1, this.user).toUri().getPath())
|
|
|
.exists());
|
|
|
-
|
|
|
+
|
|
|
dispatcher.await();
|
|
|
|
|
|
ApplicationEvent expectedEvents[] = new ApplicationEvent[]{
|
|
@@ -283,7 +294,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
|
|
|
this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
|
|
|
this.remoteRootLogDir.getAbsolutePath());
|
|
|
-
|
|
|
+ String[] fileNames = new String[] { "stdout", "stderr", "syslog" };
|
|
|
DrainDispatcher dispatcher = createDispatcher();
|
|
|
EventHandler<ApplicationEvent> appEventHandler = mock(EventHandler.class);
|
|
|
dispatcher.register(ApplicationEventType.class, appEventHandler);
|
|
@@ -310,7 +321,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
ContainerId container11 = BuilderUtils.newContainerId(appAttemptId1, 1);
|
|
|
|
|
|
// Simulate log-file creation
|
|
|
- writeContainerLogs(app1LogDir, container11);
|
|
|
+ writeContainerLogs(app1LogDir, container11, fileNames);
|
|
|
logAggregationService.handle(
|
|
|
new LogHandlerContainerFinishedEvent(container11, 0));
|
|
|
|
|
@@ -328,13 +339,13 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
|
|
|
ContainerId container21 = BuilderUtils.newContainerId(appAttemptId2, 1);
|
|
|
|
|
|
- writeContainerLogs(app2LogDir, container21);
|
|
|
+ writeContainerLogs(app2LogDir, container21, fileNames);
|
|
|
logAggregationService.handle(
|
|
|
new LogHandlerContainerFinishedEvent(container21, 0));
|
|
|
|
|
|
ContainerId container12 = BuilderUtils.newContainerId(appAttemptId1, 2);
|
|
|
|
|
|
- writeContainerLogs(app1LogDir, container12);
|
|
|
+ writeContainerLogs(app1LogDir, container12, fileNames);
|
|
|
logAggregationService.handle(
|
|
|
new LogHandlerContainerFinishedEvent(container12, 0));
|
|
|
|
|
@@ -365,22 +376,22 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
reset(appEventHandler);
|
|
|
|
|
|
ContainerId container31 = BuilderUtils.newContainerId(appAttemptId3, 1);
|
|
|
- writeContainerLogs(app3LogDir, container31);
|
|
|
+ writeContainerLogs(app3LogDir, container31, fileNames);
|
|
|
logAggregationService.handle(
|
|
|
new LogHandlerContainerFinishedEvent(container31, 0));
|
|
|
|
|
|
ContainerId container32 = BuilderUtils.newContainerId(appAttemptId3, 2);
|
|
|
- writeContainerLogs(app3LogDir, container32);
|
|
|
+ writeContainerLogs(app3LogDir, container32, fileNames);
|
|
|
logAggregationService.handle(
|
|
|
new LogHandlerContainerFinishedEvent(container32, 1)); // Failed
|
|
|
|
|
|
ContainerId container22 = BuilderUtils.newContainerId(appAttemptId2, 2);
|
|
|
- writeContainerLogs(app2LogDir, container22);
|
|
|
+ writeContainerLogs(app2LogDir, container22, fileNames);
|
|
|
logAggregationService.handle(
|
|
|
new LogHandlerContainerFinishedEvent(container22, 0));
|
|
|
|
|
|
ContainerId container33 = BuilderUtils.newContainerId(appAttemptId3, 3);
|
|
|
- writeContainerLogs(app3LogDir, container33);
|
|
|
+ writeContainerLogs(app3LogDir, container33, fileNames);
|
|
|
logAggregationService.handle(
|
|
|
new LogHandlerContainerFinishedEvent(container33, 0));
|
|
|
|
|
@@ -395,11 +406,13 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
assertEquals(0, logAggregationService.getNumAggregators());
|
|
|
|
|
|
verifyContainerLogs(logAggregationService, application1,
|
|
|
- new ContainerId[] { container11, container12 });
|
|
|
+ new ContainerId[] { container11, container12 }, fileNames, 3, false);
|
|
|
+
|
|
|
verifyContainerLogs(logAggregationService, application2,
|
|
|
- new ContainerId[] { container21 });
|
|
|
+ new ContainerId[] { container21 }, fileNames, 3, false);
|
|
|
+
|
|
|
verifyContainerLogs(logAggregationService, application3,
|
|
|
- new ContainerId[] { container31, container32 });
|
|
|
+ new ContainerId[] { container31, container32 }, fileNames, 3, false);
|
|
|
|
|
|
dispatcher.await();
|
|
|
|
|
@@ -591,7 +604,8 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
doThrow(new YarnRuntimeException("KABOOM!"))
|
|
|
.when(logAggregationService).initAppAggregator(
|
|
|
eq(appId), eq(user), any(Credentials.class),
|
|
|
- any(ContainerLogsRetentionPolicy.class), anyMap());
|
|
|
+ any(ContainerLogsRetentionPolicy.class), anyMap(),
|
|
|
+ any(LogAggregationContext.class));
|
|
|
|
|
|
logAggregationService.handle(new LogHandlerAppStartedEvent(appId,
|
|
|
this.user, null,
|
|
@@ -672,26 +686,62 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
assertEquals(0, logAggregationService.getNumAggregators());
|
|
|
}
|
|
|
|
|
|
- private void writeContainerLogs(File appLogDir, ContainerId containerId)
|
|
|
- throws IOException {
|
|
|
+ private void writeContainerLogs(File appLogDir, ContainerId containerId,
|
|
|
+ String[] fileName) throws IOException {
|
|
|
// ContainerLogDir should be created
|
|
|
String containerStr = ConverterUtils.toString(containerId);
|
|
|
File containerLogDir = new File(appLogDir, containerStr);
|
|
|
containerLogDir.mkdir();
|
|
|
- for (String fileType : new String[] { "stdout", "stderr", "syslog" }) {
|
|
|
+ for (String fileType : fileName) {
|
|
|
Writer writer11 = new FileWriter(new File(containerLogDir, fileType));
|
|
|
writer11.write(containerStr + " Hello " + fileType + "!");
|
|
|
writer11.close();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void verifyContainerLogs(
|
|
|
- LogAggregationService logAggregationService, ApplicationId appId,
|
|
|
- ContainerId[] expectedContainerIds) throws IOException {
|
|
|
+ private void verifyContainerLogs(LogAggregationService logAggregationService,
|
|
|
+ ApplicationId appId, ContainerId[] expectedContainerIds,
|
|
|
+ String[] logFiles, int numOfContainerLogs, boolean multiLogs)
|
|
|
+ throws IOException {
|
|
|
+ Path appLogDir = logAggregationService.getRemoteAppLogDir(appId, this.user);
|
|
|
+ RemoteIterator<FileStatus> nodeFiles = null;
|
|
|
+ try {
|
|
|
+ Path qualifiedLogDir =
|
|
|
+ FileContext.getFileContext(this.conf).makeQualified(appLogDir);
|
|
|
+ nodeFiles =
|
|
|
+ FileContext.getFileContext(qualifiedLogDir.toUri(), this.conf)
|
|
|
+ .listStatus(appLogDir);
|
|
|
+ } catch (FileNotFoundException fnf) {
|
|
|
+ Assert.fail("Should have log files");
|
|
|
+ }
|
|
|
+
|
|
|
+ Assert.assertTrue(nodeFiles.hasNext());
|
|
|
+ FileStatus targetNodeFile = null;
|
|
|
+ if (! multiLogs) {
|
|
|
+ targetNodeFile = nodeFiles.next();
|
|
|
+ Assert.assertTrue(targetNodeFile.getPath().getName().equals(
|
|
|
+ LogAggregationUtils.getNodeString(logAggregationService.getNodeId())));
|
|
|
+ } else {
|
|
|
+ long fileCreateTime = 0;
|
|
|
+ while (nodeFiles.hasNext()) {
|
|
|
+ FileStatus nodeFile = nodeFiles.next();
|
|
|
+ if (!nodeFile.getPath().getName()
|
|
|
+ .contains(LogAggregationUtils.TMP_FILE_SUFFIX)) {
|
|
|
+ long time =
|
|
|
+ Long.parseLong(nodeFile.getPath().getName().split("_")[2]);
|
|
|
+ if (time > fileCreateTime) {
|
|
|
+ targetNodeFile = nodeFile;
|
|
|
+ fileCreateTime = time;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ String[] fileName = targetNodeFile.getPath().getName().split("_");
|
|
|
+ Assert.assertTrue(fileName.length == 3);
|
|
|
+ Assert.assertEquals(fileName[0] + ":" + fileName[1],
|
|
|
+ logAggregationService.getNodeId().toString());
|
|
|
+ }
|
|
|
AggregatedLogFormat.LogReader reader =
|
|
|
- new AggregatedLogFormat.LogReader(this.conf,
|
|
|
- logAggregationService.getRemoteNodeLogFileForApp(appId, this.user));
|
|
|
-
|
|
|
+ new AggregatedLogFormat.LogReader(this.conf, targetNodeFile.getPath());
|
|
|
Assert.assertEquals(this.user, reader.getApplicationOwner());
|
|
|
verifyAcls(reader.getApplicationAcls());
|
|
|
|
|
@@ -749,8 +799,8 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
for (ContainerId cId : expectedContainerIds) {
|
|
|
String containerStr = ConverterUtils.toString(cId);
|
|
|
Map<String, String> thisContainerMap = logMap.remove(containerStr);
|
|
|
- Assert.assertEquals(3, thisContainerMap.size());
|
|
|
- for (String fileType : new String[] { "stdout", "stderr", "syslog" }) {
|
|
|
+ Assert.assertEquals(numOfContainerLogs, thisContainerMap.size());
|
|
|
+ for (String fileType : logFiles) {
|
|
|
String expectedValue = containerStr + " Hello " + fileType + "!";
|
|
|
LOG.info("Expected log-content : " + new String(expectedValue));
|
|
|
String foundValue = thisContainerMap.remove(fileType);
|
|
@@ -987,4 +1037,331 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
sb.append("]");
|
|
|
return sb.toString();
|
|
|
}
|
|
|
+
|
|
|
+ @Test (timeout = 50000)
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ public void testLogAggregationServiceWithPatterns() throws Exception {
|
|
|
+
|
|
|
+ LogAggregationContext logAggregationContextWithIncludePatterns =
|
|
|
+ Records.newRecord(LogAggregationContext.class);
|
|
|
+ String includePattern = "stdout|syslog";
|
|
|
+ logAggregationContextWithIncludePatterns.setIncludePattern(includePattern);
|
|
|
+
|
|
|
+ LogAggregationContext LogAggregationContextWithExcludePatterns =
|
|
|
+ Records.newRecord(LogAggregationContext.class);
|
|
|
+ String excludePattern = "stdout|syslog";
|
|
|
+ LogAggregationContextWithExcludePatterns.setExcludePattern(excludePattern);
|
|
|
+
|
|
|
+ this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
|
|
|
+ this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
|
|
|
+ this.remoteRootLogDir.getAbsolutePath());
|
|
|
+
|
|
|
+ DrainDispatcher dispatcher = createDispatcher();
|
|
|
+ EventHandler<ApplicationEvent> appEventHandler = mock(EventHandler.class);
|
|
|
+ dispatcher.register(ApplicationEventType.class, appEventHandler);
|
|
|
+
|
|
|
+ ApplicationId application1 = BuilderUtils.newApplicationId(1234, 1);
|
|
|
+ ApplicationId application2 = BuilderUtils.newApplicationId(1234, 2);
|
|
|
+ ApplicationId application3 = BuilderUtils.newApplicationId(1234, 3);
|
|
|
+ ApplicationId application4 = BuilderUtils.newApplicationId(1234, 4);
|
|
|
+ Application mockApp = mock(Application.class);
|
|
|
+ when(mockApp.getContainers()).thenReturn(
|
|
|
+ new HashMap<ContainerId, Container>());
|
|
|
+
|
|
|
+ this.context.getApplications().put(application1, mockApp);
|
|
|
+ this.context.getApplications().put(application2, mockApp);
|
|
|
+ this.context.getApplications().put(application3, mockApp);
|
|
|
+ this.context.getApplications().put(application4, mockApp);
|
|
|
+
|
|
|
+ LogAggregationService logAggregationService =
|
|
|
+ new LogAggregationService(dispatcher, this.context, this.delSrvc,
|
|
|
+ super.dirsHandler);
|
|
|
+ logAggregationService.init(this.conf);
|
|
|
+ logAggregationService.start();
|
|
|
+
|
|
|
+ // LogContext for application1 has includePatten which includes
|
|
|
+ // stdout and syslog.
|
|
|
+ // After logAggregation is finished, we expect the logs for application1
|
|
|
+ // has only logs from stdout and syslog
|
|
|
+ // AppLogDir should be created
|
|
|
+ File appLogDir1 =
|
|
|
+ new File(localLogDir, ConverterUtils.toString(application1));
|
|
|
+ appLogDir1.mkdir();
|
|
|
+ logAggregationService.handle(new LogHandlerAppStartedEvent(application1,
|
|
|
+ this.user, null, ContainerLogsRetentionPolicy.ALL_CONTAINERS, this.acls,
|
|
|
+ logAggregationContextWithIncludePatterns));
|
|
|
+
|
|
|
+ ApplicationAttemptId appAttemptId1 =
|
|
|
+ BuilderUtils.newApplicationAttemptId(application1, 1);
|
|
|
+ ContainerId container1 = BuilderUtils.newContainerId(appAttemptId1, 1);
|
|
|
+
|
|
|
+ // Simulate log-file creation
|
|
|
+ writeContainerLogs(appLogDir1, container1, new String[] { "stdout",
|
|
|
+ "stderr", "syslog" });
|
|
|
+ logAggregationService.handle(new LogHandlerContainerFinishedEvent(
|
|
|
+ container1, 0));
|
|
|
+
|
|
|
+ // LogContext for application2 has excludePatten which includes
|
|
|
+ // stdout and syslog.
|
|
|
+ // After logAggregation is finished, we expect the logs for application2
|
|
|
+ // has only logs from stderr
|
|
|
+ ApplicationAttemptId appAttemptId2 =
|
|
|
+ BuilderUtils.newApplicationAttemptId(application2, 1);
|
|
|
+
|
|
|
+ File app2LogDir =
|
|
|
+ new File(localLogDir, ConverterUtils.toString(application2));
|
|
|
+ app2LogDir.mkdir();
|
|
|
+ logAggregationService.handle(new LogHandlerAppStartedEvent(application2,
|
|
|
+ this.user, null, ContainerLogsRetentionPolicy.APPLICATION_MASTER_ONLY,
|
|
|
+ this.acls, LogAggregationContextWithExcludePatterns));
|
|
|
+ ContainerId container2 = BuilderUtils.newContainerId(appAttemptId2, 1);
|
|
|
+
|
|
|
+ writeContainerLogs(app2LogDir, container2, new String[] { "stdout",
|
|
|
+ "stderr", "syslog" });
|
|
|
+ logAggregationService.handle(
|
|
|
+ new LogHandlerContainerFinishedEvent(container2, 0));
|
|
|
+
|
|
|
+ // LogContext for application3 has includePattern which is *.log and
|
|
|
+ // excludePatten which includes std.log and sys.log.
|
|
|
+ // After logAggregation is finished, we expect the logs for application3
|
|
|
+ // has all logs whose suffix is .log but excluding sys.log and std.log
|
|
|
+ LogAggregationContext context1 =
|
|
|
+ Records.newRecord(LogAggregationContext.class);
|
|
|
+ context1.setIncludePattern(".*.log");
|
|
|
+ context1.setExcludePattern("sys.log|std.log");
|
|
|
+ ApplicationAttemptId appAttemptId3 =
|
|
|
+ BuilderUtils.newApplicationAttemptId(application3, 1);
|
|
|
+ File app3LogDir =
|
|
|
+ new File(localLogDir, ConverterUtils.toString(application3));
|
|
|
+ app3LogDir.mkdir();
|
|
|
+ logAggregationService.handle(new LogHandlerAppStartedEvent(application3,
|
|
|
+ this.user, null, ContainerLogsRetentionPolicy.APPLICATION_MASTER_ONLY,
|
|
|
+ this.acls, context1));
|
|
|
+ ContainerId container3 = BuilderUtils.newContainerId(appAttemptId3, 1);
|
|
|
+ writeContainerLogs(app3LogDir, container3, new String[] { "stdout",
|
|
|
+ "sys.log", "std.log", "out.log", "err.log", "log" });
|
|
|
+ logAggregationService.handle(
|
|
|
+ new LogHandlerContainerFinishedEvent(container3, 0));
|
|
|
+
|
|
|
+ // LogContext for application4 has includePattern
|
|
|
+ // which includes std.log and sys.log and
|
|
|
+ // excludePatten which includes std.log.
|
|
|
+ // After logAggregation is finished, we expect the logs for application4
|
|
|
+ // only has sys.log
|
|
|
+ LogAggregationContext context2 =
|
|
|
+ Records.newRecord(LogAggregationContext.class);
|
|
|
+ context2.setIncludePattern("sys.log|std.log");
|
|
|
+ context2.setExcludePattern("std.log");
|
|
|
+ ApplicationAttemptId appAttemptId4 =
|
|
|
+ BuilderUtils.newApplicationAttemptId(application4, 1);
|
|
|
+ File app4LogDir =
|
|
|
+ new File(localLogDir, ConverterUtils.toString(application4));
|
|
|
+ app4LogDir.mkdir();
|
|
|
+ logAggregationService.handle(new LogHandlerAppStartedEvent(application4,
|
|
|
+ this.user, null, ContainerLogsRetentionPolicy.APPLICATION_MASTER_ONLY,
|
|
|
+ this.acls, context2));
|
|
|
+ ContainerId container4 = BuilderUtils.newContainerId(appAttemptId4, 1);
|
|
|
+ writeContainerLogs(app4LogDir, container4, new String[] { "stdout",
|
|
|
+ "sys.log", "std.log", "out.log", "err.log", "log" });
|
|
|
+ logAggregationService.handle(
|
|
|
+ new LogHandlerContainerFinishedEvent(container4, 0));
|
|
|
+
|
|
|
+ dispatcher.await();
|
|
|
+ ApplicationEvent expectedInitEvents[] =
|
|
|
+ new ApplicationEvent[] { new ApplicationEvent(application1,
|
|
|
+ ApplicationEventType.APPLICATION_LOG_HANDLING_INITED),
|
|
|
+ new ApplicationEvent(application2,
|
|
|
+ ApplicationEventType.APPLICATION_LOG_HANDLING_INITED),
|
|
|
+ new ApplicationEvent(application3,
|
|
|
+ ApplicationEventType.APPLICATION_LOG_HANDLING_INITED),
|
|
|
+ new ApplicationEvent(application4,
|
|
|
+ ApplicationEventType.APPLICATION_LOG_HANDLING_INITED)};
|
|
|
+ checkEvents(appEventHandler, expectedInitEvents, false, "getType",
|
|
|
+ "getApplicationID");
|
|
|
+ reset(appEventHandler);
|
|
|
+
|
|
|
+ logAggregationService.handle(new LogHandlerAppFinishedEvent(application1));
|
|
|
+ logAggregationService.handle(new LogHandlerAppFinishedEvent(application2));
|
|
|
+ logAggregationService.handle(new LogHandlerAppFinishedEvent(application3));
|
|
|
+ logAggregationService.handle(new LogHandlerAppFinishedEvent(application4));
|
|
|
+ logAggregationService.stop();
|
|
|
+ assertEquals(0, logAggregationService.getNumAggregators());
|
|
|
+
|
|
|
+ String[] logFiles = new String[] { "stdout", "syslog" };
|
|
|
+ verifyContainerLogs(logAggregationService, application1,
|
|
|
+ new ContainerId[] { container1 }, logFiles, 2, false);
|
|
|
+
|
|
|
+ logFiles = new String[] { "stderr" };
|
|
|
+ verifyContainerLogs(logAggregationService, application2,
|
|
|
+ new ContainerId[] { container2 }, logFiles, 1, false);
|
|
|
+
|
|
|
+ logFiles = new String[] { "out.log", "err.log" };
|
|
|
+ verifyContainerLogs(logAggregationService, application3,
|
|
|
+ new ContainerId[] { container3 }, logFiles, 2, false);
|
|
|
+
|
|
|
+ logFiles = new String[] { "sys.log" };
|
|
|
+ verifyContainerLogs(logAggregationService, application4,
|
|
|
+ new ContainerId[] { container4 }, logFiles, 1, false);
|
|
|
+
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ ApplicationEvent[] expectedFinishedEvents =
|
|
|
+ new ApplicationEvent[] { new ApplicationEvent(application1,
|
|
|
+ ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED),
|
|
|
+ new ApplicationEvent(application2,
|
|
|
+ ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED),
|
|
|
+ new ApplicationEvent(application3,
|
|
|
+ ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED),
|
|
|
+ new ApplicationEvent(application4,
|
|
|
+ ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED) };
|
|
|
+ checkEvents(appEventHandler, expectedFinishedEvents, false, "getType",
|
|
|
+ "getApplicationID");
|
|
|
+ dispatcher.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ @Test (timeout = 50000)
|
|
|
+ public void testLogAggregationServiceWithInterval() throws Exception {
|
|
|
+ final int maxAttempts = 50;
|
|
|
+ LogAggregationContext logAggregationContextWithInterval =
|
|
|
+ Records.newRecord(LogAggregationContext.class);
|
|
|
+ logAggregationContextWithInterval.setRollingIntervalSeconds(5000);
|
|
|
+
|
|
|
+ this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
|
|
|
+ this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
|
|
|
+ this.remoteRootLogDir.getAbsolutePath());
|
|
|
+ // by setting this configuration, the log files will not be deleted immediately after
|
|
|
+ // they are aggregated to remote directory.
|
|
|
+ // We could use it to test whether the previous aggregated log files will be aggregated
|
|
|
+ // again in next cycle.
|
|
|
+ this.conf.setLong(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600);
|
|
|
+
|
|
|
+ DrainDispatcher dispatcher = createDispatcher();
|
|
|
+ EventHandler<ApplicationEvent> appEventHandler = mock(EventHandler.class);
|
|
|
+ dispatcher.register(ApplicationEventType.class, appEventHandler);
|
|
|
+
|
|
|
+ ApplicationId application = BuilderUtils.newApplicationId(123456, 1);
|
|
|
+ ApplicationAttemptId appAttemptId =
|
|
|
+ BuilderUtils.newApplicationAttemptId(application, 1);
|
|
|
+ ContainerId container = BuilderUtils.newContainerId(appAttemptId, 1);
|
|
|
+
|
|
|
+ Context context = spy(this.context);
|
|
|
+ ConcurrentMap<ApplicationId, Application> maps =
|
|
|
+ new ConcurrentHashMap<ApplicationId, Application>();
|
|
|
+ Application app = mock(Application.class);
|
|
|
+ Map<ContainerId, Container> containers = new HashMap<ContainerId, Container>();
|
|
|
+ containers.put(container, mock(Container.class));
|
|
|
+ maps.put(application, app);
|
|
|
+ when(app.getContainers()).thenReturn(containers);
|
|
|
+ when(context.getApplications()).thenReturn(maps);
|
|
|
+
|
|
|
+ LogAggregationService logAggregationService =
|
|
|
+ new LogAggregationService(dispatcher, context, this.delSrvc,
|
|
|
+ super.dirsHandler);
|
|
|
+
|
|
|
+ logAggregationService.init(this.conf);
|
|
|
+ logAggregationService.start();
|
|
|
+
|
|
|
+ // AppLogDir should be created
|
|
|
+ File appLogDir =
|
|
|
+ new File(localLogDir, ConverterUtils.toString(application));
|
|
|
+ appLogDir.mkdir();
|
|
|
+ logAggregationService.handle(new LogHandlerAppStartedEvent(application,
|
|
|
+ this.user, null, ContainerLogsRetentionPolicy.ALL_CONTAINERS, this.acls,
|
|
|
+ logAggregationContextWithInterval));
|
|
|
+
|
|
|
+ // Simulate log-file creation
|
|
|
+ String[] logFiles1 = new String[] { "stdout", "stderr", "syslog" };
|
|
|
+ writeContainerLogs(appLogDir, container, logFiles1);
|
|
|
+
|
|
|
+ // Do log aggregation
|
|
|
+ AppLogAggregatorImpl aggregator =
|
|
|
+ (AppLogAggregatorImpl) logAggregationService.getAppLogAggregators()
|
|
|
+ .get(application);
|
|
|
+ aggregator.doLogAggregationOutOfBand();
|
|
|
+
|
|
|
+ int count = 0;
|
|
|
+ while (numOfLogsAvailable(logAggregationService, application) != 1
|
|
|
+ && count <= maxAttempts) {
|
|
|
+ Thread.sleep(100);
|
|
|
+ count++;
|
|
|
+ }
|
|
|
+ // Container logs should be uploaded
|
|
|
+ verifyContainerLogs(logAggregationService, application,
|
|
|
+ new ContainerId[] { container }, logFiles1, 3, true);
|
|
|
+
|
|
|
+ // There is no log generated at this time. Do the log aggregation again.
|
|
|
+ aggregator.doLogAggregationOutOfBand();
|
|
|
+
|
|
|
+ // Same logs will not be aggregated again.
|
|
|
+ // Only one aggregated log file in Remote file directory.
|
|
|
+ Assert.assertEquals(numOfLogsAvailable(logAggregationService, application),
|
|
|
+ 1);
|
|
|
+
|
|
|
+ // Do log aggregation
|
|
|
+ String[] logFiles2 = new String[] { "stdout_1", "stderr_1", "syslog_1" };
|
|
|
+ writeContainerLogs(appLogDir, container, logFiles2);
|
|
|
+
|
|
|
+ aggregator.doLogAggregationOutOfBand();
|
|
|
+
|
|
|
+ count = 0;
|
|
|
+ while (numOfLogsAvailable(logAggregationService, application) != 2
|
|
|
+ && count <= maxAttempts) {
|
|
|
+ Thread.sleep(100);
|
|
|
+ count ++;
|
|
|
+ }
|
|
|
+ // Container logs should be uploaded
|
|
|
+ verifyContainerLogs(logAggregationService, application,
|
|
|
+ new ContainerId[] { container }, logFiles2, 3, true);
|
|
|
+
|
|
|
+ // create another logs
|
|
|
+ String[] logFiles3 = new String[] { "stdout_2", "stderr_2", "syslog_2" };
|
|
|
+ writeContainerLogs(appLogDir, container, logFiles3);
|
|
|
+
|
|
|
+ logAggregationService.handle(
|
|
|
+ new LogHandlerContainerFinishedEvent(container, 0));
|
|
|
+
|
|
|
+ dispatcher.await();
|
|
|
+ logAggregationService.handle(new LogHandlerAppFinishedEvent(application));
|
|
|
+ count = 0;
|
|
|
+ while (numOfLogsAvailable(logAggregationService, application) != 3
|
|
|
+ && count <= maxAttempts) {
|
|
|
+ Thread.sleep(100);
|
|
|
+ count ++;
|
|
|
+ }
|
|
|
+
|
|
|
+ verifyContainerLogs(logAggregationService, application,
|
|
|
+ new ContainerId[] { container }, logFiles3, 3, true);
|
|
|
+ logAggregationService.stop();
|
|
|
+ assertEquals(0, logAggregationService.getNumAggregators());
|
|
|
+ dispatcher.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ private int numOfLogsAvailable(LogAggregationService logAggregationService,
|
|
|
+ ApplicationId appId) throws IOException {
|
|
|
+ Path appLogDir = logAggregationService.getRemoteAppLogDir(appId, this.user);
|
|
|
+ RemoteIterator<FileStatus> nodeFiles = null;
|
|
|
+ try {
|
|
|
+ Path qualifiedLogDir =
|
|
|
+ FileContext.getFileContext(this.conf).makeQualified(appLogDir);
|
|
|
+ nodeFiles =
|
|
|
+ FileContext.getFileContext(qualifiedLogDir.toUri(), this.conf)
|
|
|
+ .listStatus(appLogDir);
|
|
|
+ } catch (FileNotFoundException fnf) {
|
|
|
+ return -1;
|
|
|
+ }
|
|
|
+ int count = 0;
|
|
|
+ while (nodeFiles.hasNext()) {
|
|
|
+ FileStatus status = nodeFiles.next();
|
|
|
+ String filename = status.getPath().getName();
|
|
|
+ if (filename.contains(LogAggregationUtils.TMP_FILE_SUFFIX)) {
|
|
|
+ return -1;
|
|
|
+ }
|
|
|
+ if (filename.contains(LogAggregationUtils
|
|
|
+ .getNodeString(logAggregationService.getNodeId()))) {
|
|
|
+ count++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return count;
|
|
|
+ }
|
|
|
}
|