Преглед на файлове

Merge -r 1165948:1165950 from trunk to branch-0.23 to fix MAPREDUCE-2655.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1165951 13f79535-47bb-0310-9956-ffa450edef68
Arun Murthy преди 13 години
родител
ревизия
c1e1e0857d
променени са 15 файла, в които са добавени 1172 реда и са изтрити 30 реда
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 201 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NMAuditLogger.java
  3. 22 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  4. 12 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
  5. 227 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNMAuditLogger.java
  6. 48 22
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
  7. 11 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  8. 19 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  9. 34 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  10. 309 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
  11. 9 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java
  12. 11 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
  13. 17 3
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  14. 244 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java
  15. 5 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -239,6 +239,9 @@ Release 0.23.0 - Unreleased
     MAPREDUCE-2774. Add startup message to ResourceManager & NodeManager on
     startup. (Venu Gopala Rao via acmurthy) 
 
+    MAPREDUCE-2655. Add audit logs to ResourceManager and NodeManager. (Thomas
+    Graves via acmurthy)
+
   OPTIMIZATIONS
 
     MAPREDUCE-2026. Make JobTracker.getJobCounters() and

+ 201 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NMAuditLogger.java

@@ -0,0 +1,201 @@
+/**
+ * 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.server.nodemanager;
+
+import java.net.InetAddress;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+/** 
+ * Manages NodeManager audit logs.
+ *
+ * Audit log format is written as key=value pairs. Tab separated.
+ */
+public class NMAuditLogger {
+  private static final Log LOG = LogFactory.getLog(NMAuditLogger.class);
+
+  static enum Keys {USER, OPERATION, TARGET, RESULT, IP, 
+                    DESCRIPTION, APPID, CONTAINERID}
+
+  public static class AuditConstants {
+    static final String SUCCESS = "SUCCESS";
+    static final String FAILURE = "FAILURE";
+    static final String KEY_VAL_SEPARATOR = "=";
+    static final char PAIR_SEPARATOR = '\t';
+
+    // Some commonly used descriptions
+    public static final String START_CONTAINER = "Start Container Request";
+    public static final String STOP_CONTAINER = "Stop Container Request";
+    public static final String FINISH_SUCCESS_CONTAINER = "Container Finished - Succeeded";
+    public static final String FINISH_FAILED_CONTAINER = "Container Finished - Failed";
+    public static final String FINISH_KILLED_CONTAINER = "Container Finished - Killed";
+  }
+
+  /**
+   * A helper api for creating an audit log for a successful event.
+   */
+  static String createSuccessLog(String user, String operation, String target, 
+      ApplicationId appId, ContainerId containerId) {
+    StringBuilder b = new StringBuilder();
+    start(Keys.USER, user, b);
+    addRemoteIP(b);
+    add(Keys.OPERATION, operation, b);
+    add(Keys.TARGET, target ,b);
+    add(Keys.RESULT, AuditConstants.SUCCESS, b);
+    if (appId != null) {
+      add(Keys.APPID, appId.toString(), b);
+    }
+    if (containerId != null) {
+      add(Keys.CONTAINERID, containerId.toString(), b);
+    }
+    return b.toString();
+  }
+
+  /**
+   * Create a readable and parseable audit log string for a successful event.
+   *
+   * @param user User who made the service request. 
+   * @param operation Operation requested by the user
+   * @param target The target on which the operation is being performed.
+   * @param appId Application Id in which operation was performed.
+   * @param containerId Container Id in which operation was performed.
+   *
+   * <br><br>
+   * Note that the {@link NMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logSuccess(String user, String operation, String target,
+      ApplicationId appId, ContainerId containerId) {
+    if (LOG.isInfoEnabled()) {
+      LOG.info(createSuccessLog(user, operation, target, appId, containerId));
+    }
+  }
+
+  /**
+   * Create a readable and parseable audit log string for a successful event.
+   *
+   * @param user User who made the service request. 
+   * @param operation Operation requested by the user
+   * @param target The target on which the operation is being performed.
+   *
+   * <br><br>
+   * Note that the {@link NMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logSuccess(String user, String operation, String target) {
+    if (LOG.isInfoEnabled()) {
+      LOG.info(createSuccessLog(user, operation, target, null, null));
+    }
+  }
+
+  /**
+   * A helper api for creating an audit log for a failure event.
+   * This is factored out for testing purpose.
+   */
+  static String createFailureLog(String user, String operation, String target, 
+      String description, ApplicationId appId, ContainerId containerId) {
+    StringBuilder b = new StringBuilder();
+    start(Keys.USER, user, b);
+    addRemoteIP(b);
+    add(Keys.OPERATION, operation, b);
+    add(Keys.TARGET, target ,b);
+    add(Keys.RESULT, AuditConstants.FAILURE, b);
+    add(Keys.DESCRIPTION, description, b);
+    if (appId != null) {
+      add(Keys.APPID, appId.toString(), b);
+    }
+    if (containerId != null) {
+      add(Keys.CONTAINERID, containerId.toString(), b);
+    }
+    return b.toString();
+  }
+
+  /**
+   * Create a readable and parseable audit log string for a failed event.
+   *
+   * @param user User who made the service request. 
+   * @param operation Operation requested by the user.
+   * @param target The target on which the operation is being performed. 
+   * @param description Some additional information as to why the operation
+   *                    failed.
+   * @param appId ApplicationId in which operation was performed.
+   * @param containerId Container Id in which operation was performed.
+   *
+   * <br><br>
+   * Note that the {@link NMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logFailure(String user, String operation, String target, 
+      String description, ApplicationId appId, ContainerId containerId) {
+    if (LOG.isWarnEnabled()) {
+      LOG.warn(createFailureLog(user, operation, target, description, appId, containerId));
+    }
+  }
+
+  /**
+   * Create a readable and parseable audit log string for a failed event.
+   *
+   * @param user User who made the service request. 
+   * @param operation Operation requested by the user.
+   * @param target The target on which the operation is being performed. 
+   * @param description Some additional information as to why the operation
+   *                    failed.
+   *
+   * <br><br>
+   * Note that the {@link NMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logFailure(String user, String operation, 
+                         String target, String description) {
+    if (LOG.isWarnEnabled()) {
+      LOG.warn(createFailureLog(user, operation, target, description, null, null));
+    }
+  }
+
+  /**
+   * A helper api to add remote IP address
+   */
+  static void addRemoteIP(StringBuilder b) {
+    InetAddress ip = Server.getRemoteIp();
+    // ip address can be null for testcases
+    if (ip != null) {
+      add(Keys.IP, ip.getHostAddress(), b);
+    }
+  }
+
+  /**
+   * Adds the first key-val pair to the passed builder in the following format
+   * key=value
+   */
+  static void start(Keys key, String value, StringBuilder b) {
+    b.append(key.name()).append(AuditConstants.KEY_VAL_SEPARATOR).append(value);
+  }
+
+  /**
+   * Appends the key-val pair to the passed builder in the following format
+   * <pair-delim>key=value
+   */
+  static void add(Keys key, String value, StringBuilder b) {
+    b.append(AuditConstants.PAIR_SEPARATOR).append(key.name())
+     .append(AuditConstants.KEY_VAL_SEPARATOR).append(value);
+  }
+}

+ 22 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -65,6 +65,8 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerManagerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
+import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger;
+import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
 import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
@@ -266,6 +268,10 @@ public class ContainerManagerImpl extends CompositeService implements
     ContainerId containerID = launchContext.getContainerId();
     ApplicationId applicationID = containerID.getAppId();
     if (context.getContainers().putIfAbsent(containerID, container) != null) {
+      NMAuditLogger.logFailure(launchContext.getUser(), 
+          AuditConstants.START_CONTAINER, "ContainerManagerImpl",
+          "Container already running on this node!",
+          applicationID, containerID);
       throw RPCUtil.getRemoteException("Container " + containerID
           + " already is running on this node!!");
     }
@@ -281,6 +287,11 @@ public class ContainerManagerImpl extends CompositeService implements
 
     // TODO: Validate the request
     dispatcher.getEventHandler().handle(new ApplicationInitEvent(container));
+
+    NMAuditLogger.logSuccess(launchContext.getUser(), 
+        AuditConstants.START_CONTAINER, "ContainerManageImpl", 
+        applicationID, containerID);
+
     StartContainerResponse response =
         recordFactory.newRecordInstance(StartContainerResponse.class);
     response.addAllServiceResponse(auxiluaryServices.getMeta());
@@ -300,12 +311,23 @@ public class ContainerManagerImpl extends CompositeService implements
     Container container = this.context.getContainers().get(containerID);
     if (container == null) {
       LOG.warn("Trying to stop unknown container " + containerID);
+      NMAuditLogger.logFailure(container.getUser(),
+          AuditConstants.STOP_CONTAINER, "ContainerManagerImpl",
+          "Trying to stop unknown container!",
+          containerID.getAppId(), containerID);
       return response; // Return immediately.
     }
     dispatcher.getEventHandler().handle(
         new ContainerKillEvent(containerID,
             "Container killed by the ApplicationMaster."));
 
+    // user logged here not ideal since just getting user from container but
+    // request doesn't have anything and should be coming from user of AM so 
+    // should be the same or should be rejected by auth before here. 
+    NMAuditLogger.logSuccess(container.getUser(), 
+        AuditConstants.STOP_CONTAINER, "ContainerManageImpl", 
+        containerID.getAppId(), containerID);
+
     // TODO: Move this code to appropriate place once kill_container is
     // implemented.
     nodeStatusUpdater.sendOutofBandHeartBeat();

+ 12 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java

@@ -42,6 +42,8 @@ import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger;
+import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationContainerFinishedEvent;
@@ -365,18 +367,28 @@ public class ContainerImpl implements Container {
       case EXITED_WITH_SUCCESS:
         metrics.endRunningContainer();
         metrics.completedContainer();
+        NMAuditLogger.logSuccess(getUser(),
+            AuditConstants.FINISH_SUCCESS_CONTAINER, "ContainerImpl",
+            getContainerID().getAppId(), getContainerID());
         break;
       case EXITED_WITH_FAILURE:
         metrics.endRunningContainer();
         // fall through
       case LOCALIZATION_FAILED:
         metrics.failedContainer();
+        NMAuditLogger.logFailure(getUser(),
+            AuditConstants.FINISH_FAILED_CONTAINER, "ContainerImpl",
+            "Container failed with state: " + getContainerState(),
+            getContainerID().getAppId(), getContainerID());
         break;
       case CONTAINER_CLEANEDUP_AFTER_KILL:
         metrics.endRunningContainer();
         // fall through
       case NEW:
         metrics.killedContainer();
+        NMAuditLogger.logSuccess(getUser(),
+            AuditConstants.FINISH_KILLED_CONTAINER, "ContainerImpl",
+            getContainerID().getAppId(), getContainerID());
     }
 
     metrics.releaseContainer(getLaunchContext().getResource());

+ 227 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNMAuditLogger.java

@@ -0,0 +1,227 @@
+/**
+ * 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.server.nodemanager;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.TestRPC.TestImpl;
+import org.apache.hadoop.ipc.TestRPC.TestProtocol;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger;
+import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.AuditConstants;
+import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.Keys;
+
+import org.apache.hadoop.net.NetUtils;
+
+import static org.mockito.Mockito.*;
+import static junit.framework.Assert.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+
+/**
+ * Tests {@link NMAuditLogger}.
+ */
+public class TestNMAuditLogger {
+  private static final Log LOG = LogFactory.getLog(TestNMAuditLogger.class);
+  private static final String USER = "test";
+  private static final String OPERATION = "oper";
+  private static final String TARGET = "tgt";
+  private static final String DESC = "description of an audit log";
+
+  private static final ApplicationId APPID = mock(ApplicationId.class);
+  private static final ContainerId CONTAINERID = mock(ContainerId.class);
+
+  @Before
+  public void setUp() throws Exception {
+    when(APPID.toString()).thenReturn("app_1");
+    when(CONTAINERID.toString()).thenReturn("container_1");
+  }
+
+
+  /**
+   * Test the AuditLog format with key-val pair.
+   */
+  @Test  
+  public void testKeyValLogFormat() throws Exception {
+    StringBuilder actLog = new StringBuilder();
+    StringBuilder expLog = new StringBuilder();
+    // add the first k=v pair and check
+    NMAuditLogger.start(Keys.USER, USER, actLog);
+    expLog.append("USER=test");
+    assertEquals(expLog.toString(), actLog.toString());
+
+    // append another k1=v1 pair to already added k=v and test
+    NMAuditLogger.add(Keys.OPERATION, OPERATION, actLog);
+    expLog.append("\tOPERATION=oper");
+    assertEquals(expLog.toString(), actLog.toString());
+
+    // append another k1=null pair and test
+    NMAuditLogger.add(Keys.APPID, (String)null, actLog);
+    expLog.append("\tAPPID=null");
+    assertEquals(expLog.toString(), actLog.toString());
+
+    // now add the target and check of the final string
+    NMAuditLogger.add(Keys.TARGET, TARGET, actLog);
+    expLog.append("\tTARGET=tgt");
+    assertEquals(expLog.toString(), actLog.toString());
+  }
+
+
+  /**
+   * Test the AuditLog format for successful events.
+   */
+  private void testSuccessLogFormatHelper(boolean checkIP, 
+      ApplicationId appId, ContainerId containerId) {
+    // check without the IP
+    String sLog = NMAuditLogger.createSuccessLog(USER, OPERATION, TARGET,
+        appId, containerId);
+    StringBuilder expLog = new StringBuilder();
+    expLog.append("USER=test\t");
+    if (checkIP) {
+      InetAddress ip = Server.getRemoteIp();
+      expLog.append(Keys.IP.name() + "=" + ip.getHostAddress() + "\t");
+    }
+    expLog.append("OPERATION=oper\tTARGET=tgt\tRESULT=SUCCESS");
+    if (appId != null) {
+      expLog.append("\tAPPID=app_1");
+    }
+    if (containerId != null) {
+      expLog.append("\tCONTAINERID=container_1");
+    }
+    assertEquals(expLog.toString(), sLog);
+  }
+
+  /**
+   * Test the AuditLog format for successful events passing nulls.
+   */
+  private void testSuccessLogNulls(boolean checkIP) {
+    String sLog = NMAuditLogger.createSuccessLog(null, null, null,
+        null, null);
+    StringBuilder expLog = new StringBuilder();
+    expLog.append("USER=null\t");
+    if (checkIP) {
+      InetAddress ip = Server.getRemoteIp();
+      expLog.append(Keys.IP.name() + "=" + ip.getHostAddress() + "\t");
+    }
+    expLog.append("OPERATION=null\tTARGET=null\tRESULT=SUCCESS");
+    assertEquals(expLog.toString(), sLog);
+  }
+
+  /**
+   * Test the AuditLog format for successful events with the various
+   * parameters.
+   */
+  private void testSuccessLogFormat(boolean checkIP) {
+    testSuccessLogFormatHelper(checkIP, null, null);
+    testSuccessLogFormatHelper(checkIP, APPID, null);
+    testSuccessLogFormatHelper(checkIP, null, CONTAINERID);
+    testSuccessLogFormatHelper(checkIP, APPID, CONTAINERID);
+    testSuccessLogNulls(checkIP);
+  }
+
+
+  /**
+   * Test the AuditLog format for failure events.
+   */
+  private void testFailureLogFormatHelper(boolean checkIP, ApplicationId appId,
+      ContainerId containerId) {
+    String fLog =
+      NMAuditLogger.createFailureLog(USER, OPERATION, TARGET, DESC, appId,
+      containerId);
+    StringBuilder expLog = new StringBuilder();
+    expLog.append("USER=test\t");
+    if (checkIP) {
+      InetAddress ip = Server.getRemoteIp();
+      expLog.append(Keys.IP.name() + "=" + ip.getHostAddress() + "\t");
+    }
+    expLog.append("OPERATION=oper\tTARGET=tgt\tRESULT=FAILURE\t");
+    expLog.append("DESCRIPTION=description of an audit log");
+
+    if (appId != null) {
+      expLog.append("\tAPPID=app_1");
+    }
+    if (containerId != null) {
+      expLog.append("\tCONTAINERID=container_1");
+    }
+    assertEquals(expLog.toString(), fLog);
+  }
+
+  /**
+   * Test the AuditLog format for failure events with the various
+   * parameters.
+   */
+  private void testFailureLogFormat(boolean checkIP) {
+    testFailureLogFormatHelper(checkIP, null, null);
+    testFailureLogFormatHelper(checkIP, APPID, null);
+    testFailureLogFormatHelper(checkIP, null, CONTAINERID);
+    testFailureLogFormatHelper(checkIP, APPID, CONTAINERID);
+  }
+
+  /**
+   * Test {@link NMAuditLogger} without IP set.
+   */
+  @Test  
+  public void testNMAuditLoggerWithoutIP() throws Exception {
+    // test without ip
+    testSuccessLogFormat(false);
+    testFailureLogFormat(false);
+  }
+
+  /**
+   * A special extension of {@link TestImpl} RPC server with 
+   * {@link TestImpl#ping()} testing the audit logs.
+   */
+  private class MyTestRPCServer extends TestImpl {
+    @Override
+    public void ping() {
+      // test with ip set
+      testSuccessLogFormat(true);
+      testFailureLogFormat(true);
+    }
+  }
+
+  /**
+   * Test {@link NMAuditLogger} with IP set.
+   */
+  @Test  
+  public void testNMAuditLoggerWithIP() throws Exception {
+    Configuration conf = new Configuration();
+    // start the IPC server
+    Server server = RPC.getServer(new MyTestRPCServer(), "0.0.0.0", 0, conf);
+    server.start();
+
+    InetSocketAddress addr = NetUtils.getConnectAddress(server);
+
+    // Make a client connection and test the audit log
+    TestProtocol proxy = (TestProtocol)RPC.getProxy(TestProtocol.class,
+                           TestProtocol.versionID, addr, conf);
+    // Start the testcase
+    proxy.ping();
+
+    server.stop();
+  }
+}

+ 48 - 22
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.Refresh
 import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.service.AbstractService;
 
 public class AdminService extends AbstractService implements RMAdminProtocol {
@@ -113,40 +114,54 @@ public class AdminService extends AbstractService implements RMAdminProtocol {
     super.stop();
   }
 
-  private void checkAcls(String method) throws YarnRemoteException {
+  private UserGroupInformation checkAcls(String method) throws YarnRemoteException {
+    UserGroupInformation user;
     try {
-      UserGroupInformation user = UserGroupInformation.getCurrentUser();
-      if (!adminAcl.isUserAllowed(user)) {
-        LOG.warn("User " + user.getShortUserName() + " doesn't have permission" +
-        " to call '" + method + "'");
-
-        throw RPCUtil.getRemoteException(
-            new AccessControlException("User " + user.getShortUserName() + 
-                " doesn't have permission" +
-                " to call '" + method + "'")
-            );
-      }
-      
-      LOG.info("RM Admin: " + method + " invoked by user " + 
-          user.getShortUserName());
-      
+      user = UserGroupInformation.getCurrentUser();
     } catch (IOException ioe) {
       LOG.warn("Couldn't get current user", ioe);
+
+      RMAuditLogger.logFailure("UNKNOWN", method,
+          adminAcl.toString(), "AdminService",
+          "Couldn't get current user");
       throw RPCUtil.getRemoteException(ioe);
     }
+
+    if (!adminAcl.isUserAllowed(user)) {
+      LOG.warn("User " + user.getShortUserName() + " doesn't have permission" +
+      " to call '" + method + "'");
+
+      RMAuditLogger.logFailure(user.getShortUserName(), method,
+          adminAcl.toString(), "AdminService",
+          AuditConstants.UNAUTHORIZED_USER);
+
+      throw RPCUtil.getRemoteException(
+          new AccessControlException("User " + user.getShortUserName() + 
+              " doesn't have permission" +
+              " to call '" + method + "'")
+          );
+    }
+    LOG.info("RM Admin: " + method + " invoked by user " + 
+        user.getShortUserName());
+      
+    return user;
   }
   
   @Override
   public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
       throws YarnRemoteException {
-    checkAcls("refreshQueues");
-    
+    UserGroupInformation user = checkAcls("refreshQueues");
     try {
       scheduler.reinitialize(conf, null, null); // ContainerTokenSecretManager can't
                                                 // be 'refreshed'
+      RMAuditLogger.logSuccess(user.getShortUserName(), "refreshQueues", 
+          "AdminService");
       return recordFactory.newRecordInstance(RefreshQueuesResponse.class);
     } catch (IOException ioe) {
       LOG.info("Exception refreshing queues ", ioe);
+      RMAuditLogger.logFailure(user.getShortUserName(), "refreshQueues",
+          adminAcl.toString(), "AdminService",
+          "Exception refreshing queues");
       throw RPCUtil.getRemoteException(ioe);
     }
   }
@@ -154,12 +169,17 @@ public class AdminService extends AbstractService implements RMAdminProtocol {
   @Override
   public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
       throws YarnRemoteException {
-    checkAcls("refreshNodes");
+    UserGroupInformation user = checkAcls("refreshNodes");
     try {
       this.nodesListManager.refreshNodes();
+      RMAuditLogger.logSuccess(user.getShortUserName(), "refreshNodes",
+          "AdminService");
       return recordFactory.newRecordInstance(RefreshNodesResponse.class);
     } catch (IOException ioe) {
       LOG.info("Exception refreshing nodes ", ioe);
+      RMAuditLogger.logFailure(user.getShortUserName(), "refreshNodes",
+          adminAcl.toString(), "AdminService",
+          "Exception refreshing nodes");
       throw RPCUtil.getRemoteException(ioe);
     }
   }
@@ -168,9 +188,11 @@ public class AdminService extends AbstractService implements RMAdminProtocol {
   public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
       RefreshSuperUserGroupsConfigurationRequest request)
       throws YarnRemoteException {
-    checkAcls("refreshSuperUserGroupsConfiguration");
+    UserGroupInformation user = checkAcls("refreshSuperUserGroupsConfiguration");
     
     ProxyUsers.refreshSuperUserGroupsConfiguration(new Configuration());
+    RMAuditLogger.logSuccess(user.getShortUserName(),
+        "refreshSuperUserGroupsConfiguration", "AdminService");
     
     return recordFactory.newRecordInstance(
         RefreshSuperUserGroupsConfigurationResponse.class);
@@ -179,9 +201,11 @@ public class AdminService extends AbstractService implements RMAdminProtocol {
   @Override
   public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
       RefreshUserToGroupsMappingsRequest request) throws YarnRemoteException {
-    checkAcls("refreshUserToGroupsMappings");
+    UserGroupInformation user = checkAcls("refreshUserToGroupsMappings");
     
     Groups.getUserToGroupsMappingService().refresh();
+    RMAuditLogger.logSuccess(user.getShortUserName(), 
+        "refreshUserToGroupsMappings", "AdminService");
 
     return recordFactory.newRecordInstance(
         RefreshUserToGroupsMappingsResponse.class);
@@ -190,12 +214,14 @@ public class AdminService extends AbstractService implements RMAdminProtocol {
   @Override
   public RefreshAdminAclsResponse refreshAdminAcls(
       RefreshAdminAclsRequest request) throws YarnRemoteException {
-    checkAcls("refreshAdminAcls");
+    UserGroupInformation user = checkAcls("refreshAdminAcls");
     
     Configuration conf = new Configuration();
     adminAcl = 
       new AccessControlList(
           conf.get(RMConfig.RM_ADMIN_ACL, RMConfig.DEFAULT_RM_ADMIN_ACL));
+    RMAuditLogger.logSuccess(user.getShortUserName(), "refreshAdminAcls", 
+        "AdminService");
 
     return recordFactory.newRecordInstance(RefreshAdminAclsResponse.class);
   }

+ 11 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRespons
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.AMResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -49,6 +50,7 @@ import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager;
 import org.apache.hadoop.yarn.security.SchedulerSecurityInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -115,11 +117,16 @@ public class ApplicationMasterService extends AbstractService implements
 
     ApplicationAttemptId applicationAttemptId = request
         .getApplicationAttemptId();
+    ApplicationId appID = applicationAttemptId.getApplicationId();
     AMResponse lastResponse = responseMap.get(applicationAttemptId);
     if (lastResponse == null) {
       String message = "Application doesn't exist in cache "
           + applicationAttemptId;
       LOG.error(message);
+      RMAuditLogger.logFailure(this.rmContext.getRMApps().get(appID).getUser(), 
+          AuditConstants.REGISTER_AM, message, "ApplicationMasterService",
+          "Error in registering application master", appID,
+          applicationAttemptId);
       throw RPCUtil.getRemoteException(message);
     }
 
@@ -133,6 +140,10 @@ public class ApplicationMasterService extends AbstractService implements
           new RMAppAttemptRegistrationEvent(applicationAttemptId, request
               .getHost(), request.getRpcPort(), request.getTrackingUrl()));
 
+      RMAuditLogger.logSuccess(this.rmContext.getRMApps().get(appID).getUser(),
+          AuditConstants.REGISTER_AM, "ApplicationMasterService", appID, 
+          applicationAttemptId);
+
       // Pick up min/max resource from scheduler...
       RegisterApplicationMasterResponse response = recordFactory
           .newRecordInstance(RegisterApplicationMasterResponse.class);

+ 19 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@@ -195,9 +196,11 @@ public class ClientRMService extends AbstractService implements
       SubmitApplicationRequest request) throws YarnRemoteException {
     ApplicationSubmissionContext submissionContext = request
         .getApplicationSubmissionContext();
+    ApplicationId applicationId = null;
+    String user = null;
     try {
-      String user = UserGroupInformation.getCurrentUser().getShortUserName();
-      ApplicationId applicationId = submissionContext.getApplicationId();
+      user = UserGroupInformation.getCurrentUser().getShortUserName();
+      applicationId = submissionContext.getApplicationId();
       if (rmContext.getRMApps().get(applicationId) != null) {
         throw new IOException("Application with id " + applicationId
             + " is already present! Cannot add a duplicate!");
@@ -207,8 +210,13 @@ public class ClientRMService extends AbstractService implements
 
       LOG.info("Application with id " + applicationId.getId() + 
           " submitted by user " + user + " with " + submissionContext);
+      RMAuditLogger.logSuccess(user, AuditConstants.SUBMIT_APP_REQUEST,
+          "ClientRMService", applicationId);
     } catch (IOException ie) {
       LOG.info("Exception in submitting application", ie);
+      RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST, 
+          ie.getMessage(), "ClientRMService",
+          "Exception in submitting application", applicationId);
       throw RPCUtil.getRemoteException(ie);
     }
 
@@ -228,6 +236,9 @@ public class ClientRMService extends AbstractService implements
       callerUGI = UserGroupInformation.getCurrentUser();
     } catch (IOException ie) {
       LOG.info("Error getting UGI ", ie);
+      RMAuditLogger.logFailure("UNKNOWN", AuditConstants.KILL_APP_REQUEST,
+          "UNKNOWN", "ClientRMService" , "Error getting UGI",
+          applicationId);
       throw RPCUtil.getRemoteException(ie);
     }
 
@@ -235,6 +246,10 @@ public class ClientRMService extends AbstractService implements
     // TODO: What if null
     if (!checkAccess(callerUGI, application.getUser(),
         ApplicationACL.MODIFY_APP)) {
+      RMAuditLogger.logFailure(callerUGI.getShortUserName(), 
+          AuditConstants.KILL_APP_REQUEST, 
+          "User doesn't have MODIFY_APP permissions", "ClientRMService",
+          AuditConstants.UNAUTHORIZED_USER, applicationId);
       throw RPCUtil.getRemoteException(new AccessControlException("User "
           + callerUGI.getShortUserName() + " cannot perform operation "
           + ApplicationACL.MODIFY_APP.name() + " on " + applicationId));
@@ -243,6 +258,8 @@ public class ClientRMService extends AbstractService implements
     this.rmContext.getDispatcher().getEventHandler().handle(
         new RMAppEvent(applicationId, RMAppEventType.KILL));
 
+    RMAuditLogger.logSuccess(callerUGI.getShortUserName(), 
+        AuditConstants.KILL_APP_REQUEST, "ClientRMService" , applicationId);
     FinishApplicationResponse response = recordFactory
         .newRecordInstance(FinishApplicationResponse.class);
     return response;

+ 34 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -32,6 +32,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier;
 import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager;
 import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -165,8 +167,39 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
       LOG.error("RMAppManager received completed appId of null, skipping");
     } else {
       completedApps.add(appId);  
+      writeAuditLog(appId);
     }
-  };
+  }
+
+  protected void writeAuditLog(ApplicationId appId) {
+    RMApp app = rmContext.getRMApps().get(appId);
+    String operation = "UNKONWN";
+    boolean success = false;
+    switch (app.getState()) {
+      case FAILED: 
+        operation = AuditConstants.FINISH_FAILED_APP;
+        break;
+      case FINISHED:
+        operation = AuditConstants.FINISH_SUCCESS_APP;
+        success = true;
+        break;
+      case KILLED: 
+        operation = AuditConstants.FINISH_KILLED_APP;
+        success = true;
+        break;
+      default:
+    }
+    
+    if (success) {
+      RMAuditLogger.logSuccess(app.getUser(), operation,
+          "RMAppManager", app.getApplicationId());
+    } else {
+      StringBuilder diag = app.getDiagnostics(); 
+      String msg = diag == null ? null : diag.toString();
+      RMAuditLogger.logFailure(app.getUser(), operation, msg, "RMAppManager",
+          "App failed with state: " + app.getState(), appId);
+    }
+  }
 
   /*
    * check to see if hit the limit for max # completed apps kept

+ 309 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java

@@ -0,0 +1,309 @@
+/**
+ * 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.server.resourcemanager;
+
+import java.net.InetAddress;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+/** 
+ * Manages ResourceManager audit logs. 
+ *
+ * Audit log format is written as key=value pairs. Tab separated.
+ */
+public class RMAuditLogger {
+  private static final Log LOG = LogFactory.getLog(RMAuditLogger.class);
+
+  static enum Keys {USER, OPERATION, TARGET, RESULT, IP, PERMISSIONS,
+                    DESCRIPTION, APPID, APPATTEMPTID, CONTAINERID}
+
+  public static class AuditConstants {
+    static final String SUCCESS = "SUCCESS";
+    static final String FAILURE = "FAILURE";
+    static final String KEY_VAL_SEPARATOR = "=";
+    static final char PAIR_SEPARATOR = '\t';
+
+    public static final String KILL_APP_REQUEST = "Kill Application Request";
+    public static final String SUBMIT_APP_REQUEST = "Submit Application Request";
+    public static final String FINISH_SUCCESS_APP = "Application Finished - Succeeded";
+    public static final String FINISH_FAILED_APP = "Application Finished - Failed";
+    public static final String FINISH_KILLED_APP = "Application Finished - Killed";
+    public static final String REGISTER_AM = "Register App Master";
+    public static final String ALLOC_CONTAINER = "AM Allocated Container";
+    public static final String RELEASE_CONTAINER = "AM Released Container";
+
+    // Some commonly used descriptions
+    public static final String UNAUTHORIZED_USER = "Unauthorized user";
+  }
+
+  /**
+   * A helper api for creating an audit log for a successful event.
+   */
+  static String createSuccessLog(String user, String operation, String target,
+      ApplicationId appId, ApplicationAttemptId attemptId, ContainerId containerId) {
+    StringBuilder b = new StringBuilder();
+    start(Keys.USER, user, b);
+    addRemoteIP(b);
+    add(Keys.OPERATION, operation, b);
+    add(Keys.TARGET, target ,b);
+    add(Keys.RESULT, AuditConstants.SUCCESS, b);
+    if (appId != null) {
+      add(Keys.APPID, appId.toString(), b);
+    }
+    if (attemptId != null) {
+      add(Keys.APPATTEMPTID, attemptId.toString(), b);
+    }
+    if (containerId != null) {
+      add(Keys.CONTAINERID, containerId.toString(), b);
+    }
+    return b.toString();
+  }
+
+  /**
+   * Create a readable and parseable audit log string for a successful event.
+   *
+   * @param user User who made the service request to the ResourceManager
+   * @param operation Operation requested by the user.
+   * @param target The target on which the operation is being performed. 
+   * @param appId Application Id in which operation was performed.
+   * @param containerId Container Id in which operation was performed.
+   *
+   * <br><br>
+   * Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logSuccess(String user, String operation, String target, 
+      ApplicationId appId, ContainerId containerId) {
+    if (LOG.isInfoEnabled()) {
+      LOG.info(createSuccessLog(user, operation, target, appId, null, 
+          containerId));
+    }
+  }
+
+  /**
+   * Create a readable and parseable audit log string for a successful event.
+   *
+   * @param user User who made the service request to the ResourceManager.
+   * @param operation Operation requested by the user.
+   * @param target The target on which the operation is being performed. 
+   * @param appId Application Id in which operation was performed.
+   * @param attemptId Application Attempt Id in which operation was performed.
+   *
+   * <br><br>
+   * Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logSuccess(String user, String operation, String target, 
+      ApplicationId appId, ApplicationAttemptId attemptId) {
+    if (LOG.isInfoEnabled()) {
+      LOG.info(createSuccessLog(user, operation, target, appId, attemptId,
+          null));
+    }
+  }
+
+
+  /**
+   * Create a readable and parseable audit log string for a successful event.
+   *
+   * @param user User who made the service request to the ResourceManager.
+   * @param operation Operation requested by the user.
+   * @param target The target on which the operation is being performed. 
+   * @param appId Application Id in which operation was performed.
+   *
+   * <br><br>
+   * Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logSuccess(String user, String operation, String target,
+      ApplicationId appId) {
+    if (LOG.isInfoEnabled()) {
+      LOG.info(createSuccessLog(user, operation, target, appId, null, null));
+    }
+  }
+
+  /**
+   * Create a readable and parseable audit log string for a successful event.
+   *
+   * @param user User who made the service request. 
+   * @param operation Operation requested by the user.
+   * @param target The target on which the operation is being performed. 
+   *
+   * <br><br>
+   * Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logSuccess(String user, String operation, String target) {
+    if (LOG.isInfoEnabled()) {
+      LOG.info(createSuccessLog(user, operation, target, null, null, null));
+    }
+  }
+
+  /**
+   * A helper api for creating an audit log for a failure event.
+   */
+  static String createFailureLog(String user, String operation, String perm,
+      String target, String description, ApplicationId appId,
+      ApplicationAttemptId attemptId, ContainerId containerId) {
+    StringBuilder b = new StringBuilder();
+    start(Keys.USER, user, b);
+    addRemoteIP(b);
+    add(Keys.OPERATION, operation, b);
+    add(Keys.TARGET, target ,b);
+    add(Keys.RESULT, AuditConstants.FAILURE, b);
+    add(Keys.DESCRIPTION, description, b);
+    add(Keys.PERMISSIONS, perm, b);
+    if (appId != null) {
+      add(Keys.APPID, appId.toString(), b);
+    }
+    if (attemptId != null) {
+      add(Keys.APPATTEMPTID, attemptId.toString(), b);
+    }
+    if (containerId != null) {
+      add(Keys.CONTAINERID, containerId.toString(), b);
+    }
+    return b.toString();
+  }
+
+  /**
+   * Create a readable and parseable audit log string for a failed event.
+   *
+   * @param user User who made the service request. 
+   * @param operation Operation requested by the user.
+   * @param perm Target permissions. 
+   * @param target The target on which the operation is being performed. 
+   * @param description Some additional information as to why the operation
+   *                    failed.
+   * @param appId Application Id in which operation was performed.
+   * @param containerId Container Id in which operation was performed.
+   *
+   * <br><br>
+   * Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logFailure(String user, String operation, String perm,
+      String target, String description, ApplicationId appId, 
+      ContainerId containerId) {
+    if (LOG.isWarnEnabled()) {
+      LOG.warn(createFailureLog(user, operation, perm, target, description,
+          appId, null, containerId));
+    }
+  }
+
+  /**
+   * Create a readable and parseable audit log string for a failed event.
+   *
+   * @param user User who made the service request. 
+   * @param operation Operation requested by the user.
+   * @param perm Target permissions.
+   * @param target The target on which the operation is being performed. 
+   * @param description Some additional information as to why the operation
+   *                    failed.
+   * @param appId ApplicationId in which operation was performed.
+   *
+   * <br><br>
+   * Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logFailure(String user, String operation, String perm,
+      String target, String description, ApplicationId appId, 
+      ApplicationAttemptId attemptId) {
+    if (LOG.isWarnEnabled()) {
+      LOG.warn(createFailureLog(user, operation, perm, target, description,
+          appId, attemptId, null));
+    }
+  }
+
+
+  /**
+   * Create a readable and parseable audit log string for a failed event.
+   *
+   * @param user User who made the service request. 
+   * @param operation Operation requested by the user.
+   * @param perm Target permissions.
+   * @param target The target on which the operation is being performed. 
+   * @param description Some additional information as to why the operation
+   *                    failed.
+   * @param appId ApplicationId in which operation was performed.
+   *
+   * <br><br>
+   * Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logFailure(String user, String operation, String perm,
+      String target, String description, ApplicationId appId) {
+    if (LOG.isWarnEnabled()) {
+      LOG.warn(createFailureLog(user, operation, perm, target, description,
+          appId, null, null));
+    }
+  }
+
+  /**
+   * Create a readable and parseable audit log string for a failed event.
+   *
+   * @param user User who made the service request.
+   * @param operation Operation requested by the user.
+   * @param perm Target permissions. 
+   * @param target The target on which the operation is being performed. 
+   * @param description Some additional information as to why the operation
+   *                    failed.
+   *
+   * <br><br>
+   * Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
+   * and hence the value fields should not contains tabs ('\t').
+   */
+  public static void logFailure(String user, String operation, String perm,
+      String target, String description) {
+    if (LOG.isWarnEnabled()) {
+      LOG.warn(createFailureLog(user, operation, perm, target, description,
+          null, null, null));
+    }
+  }
+
+  /**
+   * A helper api to add remote IP address
+   */
+  static void addRemoteIP(StringBuilder b) {
+    InetAddress ip = Server.getRemoteIp();
+    // ip address can be null for testcases
+    if (ip != null) {
+      add(Keys.IP, ip.getHostAddress(), b);
+    }
+  }
+
+  /**
+   * Adds the first key-val pair to the passed builder in the following format
+   * key=value
+   */
+  static void start(Keys key, String value, StringBuilder b) {
+    b.append(key.name()).append(AuditConstants.KEY_VAL_SEPARATOR).append(value);
+  }
+
+  /**
+   * Appends the key-val pair to the passed builder in the following format
+   * <pair-delim>key=value
+   */
+  static void add(Keys key, String value, StringBuilder b) {
+    b.append(AuditConstants.PAIR_SEPARATOR).append(key.name())
+     .append(AuditConstants.KEY_VAL_SEPARATOR).append(value);
+  }
+}

+ 9 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java

@@ -39,6 +39,8 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@@ -182,6 +184,10 @@ public class SchedulerApp {
     
     // Remove from the list of containers
     liveContainers.remove(rmContainer.getContainerId());
+
+    RMAuditLogger.logSuccess(getUser(), 
+        AuditConstants.RELEASE_CONTAINER, "SchedulerApp", 
+        getApplicationId(), containerId);
     
     // Update usage metrics 
     Resource containerResource = rmContainer.getContainer().getResource();
@@ -217,6 +223,9 @@ public class SchedulerApp {
           + " container=" + container.getId() + " host="
           + container.getNodeId().getHost() + " type=" + type);
     }
+    RMAuditLogger.logSuccess(getUser(), 
+        AuditConstants.ALLOC_CONTAINER, "SchedulerApp", 
+        getApplicationId(), container.getId());
 
     // Add it to allContainers list.
     newlyAllocatedContainers.add(rmContainer);

+ 11 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -46,6 +46,8 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
@@ -433,8 +435,15 @@ implements ResourceScheduler, CapacitySchedulerContext {
 
     // Release containers
     for (ContainerId releasedContainerId : release) {
-      completedContainer(getRMContainer(releasedContainerId), 
-          RMContainerEventType.RELEASED);
+      RMContainer rmContainer = getRMContainer(releasedContainerId);
+      if (rmContainer == null) {
+         RMAuditLogger.logFailure(application.getUser(),
+             AuditConstants.RELEASE_CONTAINER, 
+             "Unauthorized access or invalid container", "CapacityScheduler",
+             "Trying to release container not owned by app or with invalid id",
+             application.getApplicationId(), releasedContainerId);
+      }
+      completedContainer(rmContainer, RMContainerEventType.RELEASED);
     }
 
     synchronized (application) {

+ 17 - 3
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -55,6 +55,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
@@ -225,8 +227,15 @@ public class FifoScheduler implements ResourceScheduler {
 
     // Release containers
     for (ContainerId releasedContainer : release) {
-      containerCompleted(getRMContainer(releasedContainer), 
-          RMContainerEventType.RELEASED);
+      RMContainer rmContainer = getRMContainer(releasedContainer);
+      if (rmContainer == null) {
+         RMAuditLogger.logFailure(application.getUser(),
+             AuditConstants.RELEASE_CONTAINER, 
+             "Unauthorized access or invalid container", "FifoScheduler", 
+             "Trying to release container not owned by app or with invalid id",
+             application.getApplicationId(), releasedContainer);
+      }
+      containerCompleted(rmContainer, RMContainerEventType.RELEASED);
     }
 
     if (!ask.isEmpty()) {
@@ -642,6 +651,11 @@ public class FifoScheduler implements ResourceScheduler {
   @Lock(FifoScheduler.class)
   private synchronized void containerCompleted(RMContainer rmContainer,
       RMContainerEventType event) {
+    if (rmContainer == null) {
+      LOG.info("Null container completed...");
+      return;
+    }
+
     // Get the application for the finished container
     Container container = rmContainer.getContainer();
     ApplicationAttemptId applicationAttemptId = container.getId().getAppAttemptId();
@@ -725,7 +739,7 @@ public class FifoScheduler implements ResourceScheduler {
   private RMContainer getRMContainer(ContainerId containerId) {
     SchedulerApp application = 
         getApplication(containerId.getAppAttemptId());
-    return application.getRMContainer(containerId);
+    return (application == null) ? null : application.getRMContainer(containerId);
   }
 
 }

+ 244 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java

@@ -0,0 +1,244 @@
+/**
+ * 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.server.resourcemanager;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.TestRPC.TestImpl;
+import org.apache.hadoop.ipc.TestRPC.TestProtocol;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.Keys;
+
+import org.apache.hadoop.net.NetUtils;
+
+import static org.mockito.Mockito.*;
+import static junit.framework.Assert.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+
+/**
+ * Tests {@link RMAuditLogger}.
+ */
+public class TestRMAuditLogger {
+  private static final Log LOG = LogFactory.getLog(TestRMAuditLogger.class);
+  private static final String USER = "test";
+  private static final String OPERATION = "oper";
+  private static final String TARGET = "tgt";
+  private static final String PERM = "admin group";
+  private static final String DESC = "description of an audit log";
+  private static final ApplicationId APPID = mock(ApplicationId.class);
+  private static final ApplicationAttemptId ATTEMPTID = mock(ApplicationAttemptId.class);
+  private static final ContainerId CONTAINERID = mock(ContainerId.class);
+
+  @Before
+  public void setUp() throws Exception {
+    when(APPID.toString()).thenReturn("app_1");
+    when(ATTEMPTID.toString()).thenReturn("app_attempt_1");
+    when(CONTAINERID.toString()).thenReturn("container_1");
+  }
+
+
+  /**
+   * Test the AuditLog format with key-val pair.
+   */
+  @Test  
+  public void testKeyValLogFormat() throws Exception {
+    StringBuilder actLog = new StringBuilder();
+    StringBuilder expLog = new StringBuilder();
+    // add the first k=v pair and check
+    RMAuditLogger.start(Keys.USER, USER, actLog);
+    expLog.append("USER=test");
+    assertEquals(expLog.toString(), actLog.toString());
+
+    // append another k1=v1 pair to already added k=v and test
+    RMAuditLogger.add(Keys.OPERATION, OPERATION, actLog);
+    expLog.append("\tOPERATION=oper");
+    assertEquals(expLog.toString(), actLog.toString());
+
+    // append another k1=null pair and test
+    RMAuditLogger.add(Keys.APPID, (String)null, actLog);
+    expLog.append("\tAPPID=null");
+    assertEquals(expLog.toString(), actLog.toString());
+
+    // now add the target and check of the final string
+    RMAuditLogger.add(Keys.TARGET, TARGET, actLog);
+    expLog.append("\tTARGET=tgt");
+    assertEquals(expLog.toString(), actLog.toString());
+  }
+
+
+  /**
+   * Test the AuditLog format for successful events.
+   */
+  private void testSuccessLogFormatHelper(boolean checkIP, ApplicationId appId,
+      ApplicationAttemptId attemptId, ContainerId containerId) {
+    String sLog = RMAuditLogger.createSuccessLog(USER, OPERATION, TARGET,
+        appId, attemptId, containerId);
+    StringBuilder expLog = new StringBuilder();
+    expLog.append("USER=test\t");
+    if (checkIP) {
+      InetAddress ip = Server.getRemoteIp();
+      expLog.append(Keys.IP.name() + "=" + ip.getHostAddress() + "\t");
+    }
+    expLog.append("OPERATION=oper\tTARGET=tgt\tRESULT=SUCCESS");
+
+    if (appId != null) {
+      expLog.append("\tAPPID=app_1");
+    }
+    if (attemptId != null) {
+      expLog.append("\tAPPATTEMPTID=app_attempt_1");
+    }
+    if (containerId != null) {
+      expLog.append("\tCONTAINERID=container_1");
+    }
+    assertEquals(expLog.toString(), sLog);
+  }
+
+  /**
+   * Test the AuditLog format for successful events passing nulls.
+   */
+  private void testSuccessLogNulls(boolean checkIP) {
+    String sLog = RMAuditLogger.createSuccessLog(null, null, null, null, 
+        null, null);
+    StringBuilder expLog = new StringBuilder();
+    expLog.append("USER=null\t");
+    if (checkIP) {
+      InetAddress ip = Server.getRemoteIp();
+      expLog.append(Keys.IP.name() + "=" + ip.getHostAddress() + "\t");
+    }
+    expLog.append("OPERATION=null\tTARGET=null\tRESULT=SUCCESS");
+    assertEquals(expLog.toString(), sLog);
+  }
+
+  /**
+   * Test the AuditLog format for successful events with the various
+   * parameters.
+   */
+  private void testSuccessLogFormat(boolean checkIP) {
+    testSuccessLogFormatHelper(checkIP, null, null, null);
+    testSuccessLogFormatHelper(checkIP, APPID, null, null);
+    testSuccessLogFormatHelper(checkIP, null, null, CONTAINERID);
+    testSuccessLogFormatHelper(checkIP, null, ATTEMPTID, null);
+    testSuccessLogFormatHelper(checkIP, APPID, ATTEMPTID, null);
+    testSuccessLogFormatHelper(checkIP, APPID, null, CONTAINERID);
+    testSuccessLogFormatHelper(checkIP, null, ATTEMPTID, CONTAINERID);
+    testSuccessLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID);
+    testSuccessLogNulls(checkIP);
+  }
+
+
+  /**
+   * Test the AuditLog format for failure events.
+   */
+  private void testFailureLogFormatHelper(boolean checkIP, ApplicationId appId,
+      ApplicationAttemptId attemptId, ContainerId containerId) {
+    String fLog =
+      RMAuditLogger.createFailureLog(USER, OPERATION, PERM, TARGET, DESC,
+      appId, attemptId, containerId);
+    StringBuilder expLog = new StringBuilder();
+    expLog.append("USER=test\t");
+    if (checkIP) {
+      InetAddress ip = Server.getRemoteIp();
+      expLog.append(Keys.IP.name() + "=" + ip.getHostAddress() + "\t");
+    }
+    expLog.append("OPERATION=oper\tTARGET=tgt\tRESULT=FAILURE\t");
+    expLog.append("DESCRIPTION=description of an audit log");
+    expLog.append("\tPERMISSIONS=admin group");
+    if (appId != null) {
+      expLog.append("\tAPPID=app_1");
+    }
+    if (attemptId != null) {
+      expLog.append("\tAPPATTEMPTID=app_attempt_1");
+    }
+    if (containerId != null) {
+      expLog.append("\tCONTAINERID=container_1");
+    }
+    assertEquals(expLog.toString(), fLog);
+  }
+
+  /**
+   * Test the AuditLog format for failure events with the various
+   * parameters.
+   */
+  private void testFailureLogFormat(boolean checkIP) {
+    testFailureLogFormatHelper(checkIP, null, null, null);
+    testFailureLogFormatHelper(checkIP, APPID, null, null);
+    testFailureLogFormatHelper(checkIP, null, null, CONTAINERID);
+    testFailureLogFormatHelper(checkIP, null, ATTEMPTID, null);
+    testFailureLogFormatHelper(checkIP, APPID, ATTEMPTID, null);
+    testFailureLogFormatHelper(checkIP, APPID, null, CONTAINERID);
+    testFailureLogFormatHelper(checkIP, null, ATTEMPTID, CONTAINERID);
+    testFailureLogFormatHelper(checkIP, APPID, ATTEMPTID, CONTAINERID);
+  }
+
+  /**
+   * Test {@link RMAuditLogger} without IP set.
+   */
+  @Test  
+  public void testRMAuditLoggerWithoutIP() throws Exception {
+    // test without ip
+    testSuccessLogFormat(false);
+    testFailureLogFormat(false);
+  }
+
+  /**
+   * A special extension of {@link TestImpl} RPC server with 
+   * {@link TestImpl#ping()} testing the audit logs.
+   */
+  private class MyTestRPCServer extends TestImpl {
+    @Override
+    public void ping() {
+      // test with ip set
+      testSuccessLogFormat(true);
+      testFailureLogFormat(true);
+    }
+  }
+
+  /**
+   * Test {@link RMAuditLogger} with IP set.
+   */
+  @Test  
+  public void testRMAuditLoggerWithIP() throws Exception {
+    Configuration conf = new Configuration();
+    // start the IPC server
+    Server server = RPC.getServer(new MyTestRPCServer(), "0.0.0.0", 0, conf);
+    server.start();
+
+    InetSocketAddress addr = NetUtils.getConnectAddress(server);
+
+    // Make a client connection and test the audit log
+    TestProtocol proxy = (TestProtocol)RPC.getProxy(TestProtocol.class,
+                           TestProtocol.versionID, addr, conf);
+    // Start the testcase
+    proxy.ping();
+
+    server.stop();
+  }
+}

+ 5 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java

@@ -51,6 +51,11 @@ public class MockRMApp implements RMApp {
     user = userName;
   }
 
+  public MockRMApp(int newid, long time, RMAppState newState, String userName, String diag) {
+    this(newid, time, newState, userName);
+    this.diagnostics = new StringBuilder(diag);
+  }
+
   @Override
   public ApplicationId getApplicationId() {
     return id;