Browse Source

YARN-8273. Log aggregation does not warn if HDFS quota in target directory is exceeded (grepas via rkanter)

Robert Kanter 7 years ago
parent
commit
b22f56c471

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml

@@ -40,6 +40,10 @@
       <artifactId>hadoop-common</artifactId>
       <artifactId>hadoop-common</artifactId>
       <scope>provided</scope>
       <scope>provided</scope>
     </dependency>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs-client</artifactId>
+    </dependency>
 
 
 
 
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->

+ 12 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SecureIOUtils;
 import org.apache.hadoop.io.SecureIOUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
@@ -547,7 +548,7 @@ public class AggregatedLogFormat {
     }
     }
 
 
     @Override
     @Override
-    public void close() {
+    public void close() throws DSQuotaExceededException {
       try {
       try {
         if (writer != null) {
         if (writer != null) {
           writer.close();
           writer.close();
@@ -555,7 +556,16 @@ public class AggregatedLogFormat {
       } catch (Exception e) {
       } catch (Exception e) {
         LOG.warn("Exception closing writer", e);
         LOG.warn("Exception closing writer", e);
       } finally {
       } finally {
-        IOUtils.cleanupWithLogger(LOG, this.fsDataOStream);
+        try {
+          this.fsDataOStream.close();
+        } catch (DSQuotaExceededException e) {
+          LOG.error("Exception in closing {}",
+              this.fsDataOStream.getClass(), e);
+          throw e;
+        } catch (Throwable e) {
+          LOG.error("Exception in closing {}",
+              this.fsDataOStream.getClass(), e);
+        }
       }
       }
     }
     }
   }
   }

+ 45 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationDFSException.java

@@ -0,0 +1,45 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* 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
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* 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.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.logaggregation.filecontroller;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * This exception class indicates an issue during log aggregation.
+ */
+public class LogAggregationDFSException extends YarnException {
+
+  private static final long serialVersionUID = -6691549081090183145L;
+
+  public LogAggregationDFSException() {
+  }
+
+  public LogAggregationDFSException(String message) {
+    super(message);
+  }
+
+  public LogAggregationDFSException(Throwable cause) {
+    super(cause);
+  }
+
+  public LogAggregationDFSException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+}

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileController.java

@@ -162,8 +162,10 @@ public abstract class LogAggregationFileController {
 
 
   /**
   /**
    * Close the writer.
    * Close the writer.
+   * @throws LogAggregationDFSException if the closing of the writer fails
+   *         (for example due to HDFS quota being exceeded)
    */
    */
-  public abstract void closeWriter();
+  public abstract void closeWriter() throws LogAggregationDFSException;
 
 
   /**
   /**
    * Write the log content.
    * Write the log content.

+ 10 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.HarFs;
 import org.apache.hadoop.fs.HarFs;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -46,6 +47,7 @@ import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogWriter;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogWriter;
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationDFSException;
 import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileController;
 import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileController;
 import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext;
 import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
@@ -95,10 +97,15 @@ public class LogAggregationTFileController
   }
   }
 
 
   @Override
   @Override
-  public void closeWriter() {
+  public void closeWriter() throws LogAggregationDFSException {
     if (this.writer != null) {
     if (this.writer != null) {
-      this.writer.close();
-      this.writer = null;
+      try {
+        this.writer.close();
+      } catch (DSQuotaExceededException e) {
+        throw new LogAggregationDFSException(e);
+      } finally {
+        this.writer = null;
+      }
     }
     }
   }
   }
 
 

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestContainerLogsUtils.java

@@ -65,7 +65,7 @@ public final class TestContainerLogsUtils {
   public static void createContainerLogFileInRemoteFS(Configuration conf,
   public static void createContainerLogFileInRemoteFS(Configuration conf,
       FileSystem fs, String rootLogDir, ContainerId containerId, NodeId nodeId,
       FileSystem fs, String rootLogDir, ContainerId containerId, NodeId nodeId,
       String fileName, String user, String content,
       String fileName, String user, String content,
-      boolean deleteRemoteLogDir) throws IOException {
+      boolean deleteRemoteLogDir) throws Exception {
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
     //prepare the logs for remote directory
     //prepare the logs for remote directory
     ApplicationId appId = containerId.getApplicationAttemptId()
     ApplicationId appId = containerId.getApplicationAttemptId()
@@ -113,7 +113,7 @@ public final class TestContainerLogsUtils {
 
 
   private static void uploadContainerLogIntoRemoteDir(UserGroupInformation ugi,
   private static void uploadContainerLogIntoRemoteDir(UserGroupInformation ugi,
       Configuration configuration, List<String> rootLogDirs, NodeId nodeId,
       Configuration configuration, List<String> rootLogDirs, NodeId nodeId,
-      ContainerId containerId, Path appDir, FileSystem fs) throws IOException {
+      ContainerId containerId, Path appDir, FileSystem fs) throws Exception {
     Path path =
     Path path =
         new Path(appDir, LogAggregationUtils.getNodeString(nodeId));
         new Path(appDir, LogAggregationUtils.getNodeString(nodeId));
     LogAggregationFileControllerFactory factory
     LogAggregationFileControllerFactory factory

+ 38 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationDFSException;
 import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileController;
 import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileController;
 import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext;
 import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext;
 import org.apache.hadoop.yarn.logaggregation.filecontroller.tfile.LogAggregationTFileController;
 import org.apache.hadoop.yarn.logaggregation.filecontroller.tfile.LogAggregationTFileController;
@@ -263,7 +264,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     return params;
     return params;
   }
   }
 
 
-  private void uploadLogsForContainers(boolean appFinished) {
+  private void uploadLogsForContainers(boolean appFinished)
+      throws LogAggregationDFSException {
     if (this.logAggregationDisabled) {
     if (this.logAggregationDisabled) {
       return;
       return;
     }
     }
@@ -301,6 +303,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     logAggregationTimes++;
     logAggregationTimes++;
     String diagnosticMessage = "";
     String diagnosticMessage = "";
     boolean logAggregationSucceedInThisCycle = true;
     boolean logAggregationSucceedInThisCycle = true;
+    DeletionTask deletionTask = null;
     try {
     try {
       try {
       try {
         logAggregationFileController.initializeWriter(logControllerContext);
         logAggregationFileController.initializeWriter(logControllerContext);
@@ -327,10 +330,9 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
           uploadedLogsInThisCycle = true;
           uploadedLogsInThisCycle = true;
           List<Path> uploadedFilePathsInThisCycleList = new ArrayList<>();
           List<Path> uploadedFilePathsInThisCycleList = new ArrayList<>();
           uploadedFilePathsInThisCycleList.addAll(uploadedFilePathsInThisCycle);
           uploadedFilePathsInThisCycleList.addAll(uploadedFilePathsInThisCycle);
-          DeletionTask deletionTask = new FileDeletionTask(delService,
+          deletionTask = new FileDeletionTask(delService,
               this.userUgi.getShortUserName(), null,
               this.userUgi.getShortUserName(), null,
               uploadedFilePathsInThisCycleList);
               uploadedFilePathsInThisCycleList);
-          delService.delete(deletionTask);
         }
         }
 
 
         // This container is finished, and all its logs have been uploaded,
         // This container is finished, and all its logs have been uploaded,
@@ -356,9 +358,23 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
         logAggregationSucceedInThisCycle = false;
         logAggregationSucceedInThisCycle = false;
       }
       }
     } finally {
     } finally {
+      LogAggregationDFSException exc = null;
+      try {
+        this.logAggregationFileController.closeWriter();
+      } catch (LogAggregationDFSException e) {
+        diagnosticMessage = e.getMessage();
+        renameTemporaryLogFileFailed = true;
+        logAggregationSucceedInThisCycle = false;
+        exc = e;
+      }
+      if (logAggregationSucceedInThisCycle && deletionTask != null) {
+        delService.delete(deletionTask);
+      }
       sendLogAggregationReport(logAggregationSucceedInThisCycle,
       sendLogAggregationReport(logAggregationSucceedInThisCycle,
           diagnosticMessage, appFinished);
           diagnosticMessage, appFinished);
-      logAggregationFileController.closeWriter();
+      if (exc != null) {
+        throw exc;
+      }
     }
     }
   }
   }
 
 
@@ -413,13 +429,18 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
         diagnosticMessage, finalized);
         diagnosticMessage, finalized);
   }
   }
 
 
-  @SuppressWarnings("unchecked")
   @Override
   @Override
   public void run() {
   public void run() {
     try {
     try {
       doAppLogAggregation();
       doAppLogAggregation();
+    } catch (LogAggregationDFSException e) {
+      // if the log aggregation could not be performed due to DFS issues
+      // let's not clean up the log files, since that can result in
+      // loss of logs
+      LOG.error("Error occurred while aggregating the log for the application "
+          + appId, e);
     } catch (Exception e) {
     } catch (Exception e) {
-      // do post clean up of log directories on any exception
+      // do post clean up of log directories on any other exception
       LOG.error("Error occurred while aggregating the log for the application "
       LOG.error("Error occurred while aggregating the log for the application "
           + appId, e);
           + appId, e);
       doAppLogAggregationPostCleanUp();
       doAppLogAggregationPostCleanUp();
@@ -434,8 +455,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     }
     }
   }
   }
 
 
-  @SuppressWarnings("unchecked")
-  private void doAppLogAggregation() {
+  private void doAppLogAggregation() throws LogAggregationDFSException {
     while (!this.appFinishing.get() && !this.aborted.get()) {
     while (!this.appFinishing.get() && !this.aborted.get()) {
       synchronized(this) {
       synchronized(this) {
         try {
         try {
@@ -452,6 +472,9 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
         } catch (InterruptedException e) {
         } catch (InterruptedException e) {
           LOG.warn("PendingContainers queue is interrupted");
           LOG.warn("PendingContainers queue is interrupted");
           this.appFinishing.set(true);
           this.appFinishing.set(true);
+        } catch (LogAggregationDFSException e) {
+          this.appFinishing.set(true);
+          throw e;
         }
         }
       }
       }
     }
     }
@@ -460,10 +483,14 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
       return;
       return;
     }
     }
 
 
-    // App is finished, upload the container logs.
-    uploadLogsForContainers(true);
+    try {
+      // App is finished, upload the container logs.
+      uploadLogsForContainers(true);
 
 
-    doAppLogAggregationPostCleanUp();
+      doAppLogAggregationPostCleanUp();
+    } catch (LogAggregationDFSException e) {
+      LOG.error("Error during log aggregation", e);
+    }
 
 
     this.dispatcher.getEventHandler().handle(
     this.dispatcher.getEventHandler().handle(
         new ApplicationEvent(this.appId,
         new ApplicationEvent(this.appId,

+ 66 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestAppLogAggregatorImpl.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
 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.logaggregation.filecontroller.tfile.LogAggregationTFileController;
 import org.apache.hadoop.yarn.server.api.ContainerLogContext;
 import org.apache.hadoop.yarn.server.api.ContainerLogContext;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 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.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 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.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.recovery.NMNullStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
@@ -52,12 +55,14 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.mockito.stubbing.Answer;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
@@ -228,10 +233,15 @@ public class TestAppLogAggregatorImpl {
     config.setLong(
     config.setLong(
         YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, logRetentionSecs);
         YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, logRetentionSecs);
 
 
+    LogAggregationTFileController format = spy(
+        new LogAggregationTFileController());
+    format.initialize(config, "TFile");
+
+    Context context = createContext(config);
     final AppLogAggregatorInTest appLogAggregator =
     final AppLogAggregatorInTest appLogAggregator =
         createAppLogAggregator(appId, LOCAL_LOG_DIR.getAbsolutePath(),
         createAppLogAggregator(appId, LOCAL_LOG_DIR.getAbsolutePath(),
-            config, recoveredLogInitedTimeMillis,
-            deletionServiceWithExpectedFiles);
+            config, context, recoveredLogInitedTimeMillis,
+            deletionServiceWithExpectedFiles, format);
     appLogAggregator.startContainerLogAggregation(
     appLogAggregator.startContainerLogAggregation(
         new ContainerLogContext(containerId, ContainerType.TASK, 0));
         new ContainerLogContext(containerId, ContainerType.TASK, 0));
     // set app finished flag first
     // set app finished flag first
@@ -269,8 +279,10 @@ public class TestAppLogAggregatorImpl {
 
 
   private static AppLogAggregatorInTest createAppLogAggregator(
   private static AppLogAggregatorInTest createAppLogAggregator(
       ApplicationId applicationId, String rootLogDir,
       ApplicationId applicationId, String rootLogDir,
-      YarnConfiguration config, long recoveredLogInitedTimeMillis,
-      DeletionService deletionServiceWithFilesToExpect)
+      YarnConfiguration config, Context context,
+      long recoveredLogInitedTimeMillis,
+      DeletionService deletionServiceWithFilesToExpect,
+      LogAggregationTFileController tFileController)
       throws IOException {
       throws IOException {
 
 
     final Dispatcher dispatcher = createNullDispatcher();
     final Dispatcher dispatcher = createNullDispatcher();
@@ -284,16 +296,12 @@ public class TestAppLogAggregatorImpl {
     final LogAggregationContext logAggregationContext = null;
     final LogAggregationContext logAggregationContext = null;
     final Map<ApplicationAccessType, String> appAcls = new HashMap<>();
     final Map<ApplicationAccessType, String> appAcls = new HashMap<>();
 
 
-    final Context context = createContext(config);
     final FileContext fakeLfs = mock(FileContext.class);
     final FileContext fakeLfs = mock(FileContext.class);
     final Path remoteLogDirForApp = new Path(REMOTE_LOG_FILE.getAbsolutePath());
     final Path remoteLogDirForApp = new Path(REMOTE_LOG_FILE.getAbsolutePath());
-    LogAggregationTFileController format = spy(
-        new LogAggregationTFileController());
-    format.initialize(config, "TFile");
     return new AppLogAggregatorInTest(dispatcher, deletionService,
     return new AppLogAggregatorInTest(dispatcher, deletionService,
         config, applicationId, ugi, nodeId, dirsService,
         config, applicationId, ugi, nodeId, dirsService,
         remoteLogDirForApp, appAcls, logAggregationContext,
         remoteLogDirForApp, appAcls, logAggregationContext,
-        context, fakeLfs, recoveredLogInitedTimeMillis, format);
+        context, fakeLfs, recoveredLogInitedTimeMillis, tFileController);
   }
   }
 
 
   /**
   /**
@@ -423,4 +431,53 @@ public class TestAppLogAggregatorImpl {
       this.logValue = ArgumentCaptor.forClass(LogValue.class);
       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());
+  }
 }
 }

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java

@@ -87,7 +87,6 @@ import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.DocumentBuilderFactory;
 import java.io.File;
 import java.io.File;
-import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.io.StringReader;
 import java.io.StringReader;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
@@ -356,7 +355,7 @@ public class TestNMWebServices extends JerseyTestBase {
   }
   }
 
 
   @Test (timeout = 5000)
   @Test (timeout = 5000)
-  public void testContainerLogsWithNewAPI() throws IOException, JSONException{
+  public void testContainerLogsWithNewAPI() throws Exception {
     final ContainerId containerId = BuilderUtils.newContainerId(0, 0, 0, 0);
     final ContainerId containerId = BuilderUtils.newContainerId(0, 0, 0, 0);
     WebResource r = resource();
     WebResource r = resource();
     r = r.path("ws").path("v1").path("node").path("containers")
     r = r.path("ws").path("v1").path("node").path("containers")
@@ -365,7 +364,7 @@ public class TestNMWebServices extends JerseyTestBase {
   }
   }
 
 
   @Test (timeout = 5000)
   @Test (timeout = 5000)
-  public void testContainerLogsWithOldAPI() throws IOException, JSONException{
+  public void testContainerLogsWithOldAPI() throws Exception {
     final ContainerId containerId = BuilderUtils.newContainerId(1, 1, 0, 1);
     final ContainerId containerId = BuilderUtils.newContainerId(1, 1, 0, 1);
     WebResource r = resource();
     WebResource r = resource();
     r = r.path("ws").path("v1").path("node").path("containerlogs")
     r = r.path("ws").path("v1").path("node").path("containerlogs")
@@ -538,7 +537,7 @@ public class TestNMWebServices extends JerseyTestBase {
   }
   }
 
 
   private void testContainerLogs(WebResource r, ContainerId containerId)
   private void testContainerLogs(WebResource r, ContainerId containerId)
-      throws IOException {
+      throws Exception {
     final String containerIdStr = containerId.toString();
     final String containerIdStr = containerId.toString();
     final ApplicationAttemptId appAttemptId = containerId
     final ApplicationAttemptId appAttemptId = containerId
         .getApplicationAttemptId();
         .getApplicationAttemptId();