|
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.event.Event;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
|
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
|
|
|
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationDFSException;
|
|
|
import org.apache.hadoop.yarn.logaggregation.filecontroller.tfile.LogAggregationTFileController;
|
|
|
import org.apache.hadoop.yarn.server.api.ContainerLogContext;
|
|
|
import org.apache.hadoop.yarn.server.api.ContainerType;
|
|
@@ -42,7 +43,9 @@ 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;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.deletion.task.FileDeletionTask;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.logaggregation.tracker.NMLogAggregationStatusTracker;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
|
|
@@ -52,12 +55,14 @@ import org.junit.After;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.ArgumentCaptor;
|
|
|
+import org.mockito.Mockito;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.List;
|
|
@@ -228,10 +233,15 @@ public class TestAppLogAggregatorImpl {
|
|
|
config.setLong(
|
|
|
YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, logRetentionSecs);
|
|
|
|
|
|
+ LogAggregationTFileController format = spy(
|
|
|
+ new LogAggregationTFileController());
|
|
|
+ format.initialize(config, "TFile");
|
|
|
+
|
|
|
+ Context context = createContext(config);
|
|
|
final AppLogAggregatorInTest appLogAggregator =
|
|
|
createAppLogAggregator(appId, LOCAL_LOG_DIR.getAbsolutePath(),
|
|
|
- config, recoveredLogInitedTimeMillis,
|
|
|
- deletionServiceWithExpectedFiles);
|
|
|
+ config, context, recoveredLogInitedTimeMillis,
|
|
|
+ deletionServiceWithExpectedFiles, format);
|
|
|
appLogAggregator.startContainerLogAggregation(
|
|
|
new ContainerLogContext(containerId, ContainerType.TASK, 0));
|
|
|
// set app finished flag first
|
|
@@ -269,8 +279,10 @@ public class TestAppLogAggregatorImpl {
|
|
|
|
|
|
private static AppLogAggregatorInTest createAppLogAggregator(
|
|
|
ApplicationId applicationId, String rootLogDir,
|
|
|
- YarnConfiguration config, long recoveredLogInitedTimeMillis,
|
|
|
- DeletionService deletionServiceWithFilesToExpect)
|
|
|
+ YarnConfiguration config, Context context,
|
|
|
+ long recoveredLogInitedTimeMillis,
|
|
|
+ DeletionService deletionServiceWithFilesToExpect,
|
|
|
+ LogAggregationTFileController tFileController)
|
|
|
throws IOException {
|
|
|
|
|
|
final Dispatcher dispatcher = createNullDispatcher();
|
|
@@ -284,16 +296,12 @@ public class TestAppLogAggregatorImpl {
|
|
|
final LogAggregationContext logAggregationContext = null;
|
|
|
final Map<ApplicationAccessType, String> appAcls = new HashMap<>();
|
|
|
|
|
|
- final Context context = createContext(config);
|
|
|
final FileContext fakeLfs = mock(FileContext.class);
|
|
|
final Path remoteLogDirForApp = new Path(REMOTE_LOG_FILE.getAbsolutePath());
|
|
|
- LogAggregationTFileController format = spy(
|
|
|
- new LogAggregationTFileController());
|
|
|
- format.initialize(config, "TFile");
|
|
|
return new AppLogAggregatorInTest(dispatcher, deletionService,
|
|
|
config, applicationId, ugi, nodeId, dirsService,
|
|
|
remoteLogDirForApp, appAcls, logAggregationContext,
|
|
|
- context, fakeLfs, recoveredLogInitedTimeMillis, format);
|
|
|
+ context, fakeLfs, recoveredLogInitedTimeMillis, tFileController);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -423,4 +431,53 @@ public class TestAppLogAggregatorImpl {
|
|
|
this.logValue = ArgumentCaptor.forClass(LogValue.class);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDFSQuotaExceeded() throws Exception {
|
|
|
+
|
|
|
+ // the expectation is that no log files are deleted if the quota has
|
|
|
+ // been exceeded, since that would result in loss of logs
|
|
|
+ DeletionService deletionServiceWithExpectedFiles =
|
|
|
+ createDeletionServiceWithExpectedFile2Delete(Collections.emptySet());
|
|
|
+
|
|
|
+ final YarnConfiguration config = new YarnConfiguration();
|
|
|
+
|
|
|
+ ApplicationId appId = ApplicationId.newInstance(1357543L, 1);
|
|
|
+
|
|
|
+ // we need a LogAggregationTFileController that throws a
|
|
|
+ // LogAggregationDFSException
|
|
|
+ LogAggregationTFileController format =
|
|
|
+ Mockito.mock(LogAggregationTFileController.class);
|
|
|
+ Mockito.doThrow(new LogAggregationDFSException())
|
|
|
+ .when(format).closeWriter();
|
|
|
+
|
|
|
+ NodeManager.NMContext context = (NMContext) createContext(config);
|
|
|
+ context.setNMLogAggregationStatusTracker(
|
|
|
+ Mockito.mock(NMLogAggregationStatusTracker.class));
|
|
|
+
|
|
|
+ final AppLogAggregatorInTest appLogAggregator =
|
|
|
+ createAppLogAggregator(appId, LOCAL_LOG_DIR.getAbsolutePath(),
|
|
|
+ config, context, 1000L, deletionServiceWithExpectedFiles, format);
|
|
|
+
|
|
|
+ appLogAggregator.startContainerLogAggregation(
|
|
|
+ new ContainerLogContext(
|
|
|
+ ContainerId.newContainerId(
|
|
|
+ ApplicationAttemptId.newInstance(appId, 0), 0),
|
|
|
+ ContainerType.TASK, 0));
|
|
|
+ // set app finished flag first
|
|
|
+ appLogAggregator.finishLogAggregation();
|
|
|
+ appLogAggregator.run();
|
|
|
+
|
|
|
+ // verify that no files have been uploaded
|
|
|
+ ArgumentCaptor<LogValue> logValCaptor =
|
|
|
+ ArgumentCaptor.forClass(LogValue.class);
|
|
|
+ verify(appLogAggregator.getLogAggregationFileController()).write(
|
|
|
+ any(LogKey.class), logValCaptor.capture());
|
|
|
+ Set<String> filesUploaded = new HashSet<>();
|
|
|
+ LogValue logValue = logValCaptor.getValue();
|
|
|
+ for (File file: logValue.getPendingLogFilesToUploadForThisContainer()) {
|
|
|
+ filesUploaded.add(file.getAbsolutePath());
|
|
|
+ }
|
|
|
+ verifyFilesUploaded(filesUploaded, Collections.emptySet());
|
|
|
+ }
|
|
|
}
|