Browse Source

svn merge -c 1517097 to revert MAPREDUCE-5475 and YARN-707

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.1-beta@1517099 13f79535-47bb-0310-9956-ffa450edef68
Jason Darrell Lowe 11 years ago
parent
commit
44d6ab05f0

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

@@ -70,8 +70,6 @@ Release 2.1.1-beta - UNRELEASED
     MAPREDUCE-5476. Changed MR AM recovery code to cleanup staging-directory
     only after unregistering from the RM. (Jian He via vinodkv)
 
-    MAPREDUCE-5475. MRClientService does not verify ACLs properly (jlowe)
-
 Release 2.1.0-beta - 2013-08-22
 
   INCOMPATIBLE CHANGES

+ 22 - 42
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java

@@ -28,7 +28,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
@@ -79,8 +78,6 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
 import org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -178,22 +175,16 @@ public class MRClientService extends AbstractService
       return getBindAddress();
     }
     
-    private Job verifyAndGetJob(JobId jobID,
-        JobACL accessType) throws IOException {
+    private Job verifyAndGetJob(JobId jobID, 
+        boolean modifyAccess) throws IOException {
       Job job = appContext.getJob(jobID);
-      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-      if (!job.checkAccess(ugi, accessType)) {
-        throw new AccessControlException("User " + ugi.getShortUserName()
-            + " cannot perform operation " + accessType.name() + " on "
-            + jobID);
-      }
       return job;
     }
  
     private Task verifyAndGetTask(TaskId taskID, 
-        JobACL accessType) throws IOException {
+        boolean modifyAccess) throws IOException {
       Task task = verifyAndGetJob(taskID.getJobId(), 
-          accessType).getTask(taskID);
+          modifyAccess).getTask(taskID);
       if (task == null) {
         throw new IOException("Unknown Task " + taskID);
       }
@@ -201,9 +192,9 @@ public class MRClientService extends AbstractService
     }
 
     private TaskAttempt verifyAndGetAttempt(TaskAttemptId attemptID, 
-        JobACL accessType) throws IOException {
+        boolean modifyAccess) throws IOException {
       TaskAttempt attempt = verifyAndGetTask(attemptID.getTaskId(), 
-          accessType).getAttempt(attemptID);
+          modifyAccess).getAttempt(attemptID);
       if (attempt == null) {
         throw new IOException("Unknown TaskAttempt " + attemptID);
       }
@@ -214,7 +205,7 @@ public class MRClientService extends AbstractService
     public GetCountersResponse getCounters(GetCountersRequest request) 
       throws IOException {
       JobId jobId = request.getJobId();
-      Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
+      Job job = verifyAndGetJob(jobId, false);
       GetCountersResponse response =
         recordFactory.newRecordInstance(GetCountersResponse.class);
       response.setCounters(TypeConverter.toYarn(job.getAllCounters()));
@@ -225,7 +216,7 @@ public class MRClientService extends AbstractService
     public GetJobReportResponse getJobReport(GetJobReportRequest request) 
       throws IOException {
       JobId jobId = request.getJobId();
-      Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
+      Job job = verifyAndGetJob(jobId, false);
       GetJobReportResponse response = 
         recordFactory.newRecordInstance(GetJobReportResponse.class);
       if (job != null) {
@@ -244,7 +235,7 @@ public class MRClientService extends AbstractService
       GetTaskAttemptReportResponse response =
         recordFactory.newRecordInstance(GetTaskAttemptReportResponse.class);
       response.setTaskAttemptReport(
-          verifyAndGetAttempt(taskAttemptId, JobACL.VIEW_JOB).getReport());
+          verifyAndGetAttempt(taskAttemptId, false).getReport());
       return response;
     }
 
@@ -254,8 +245,7 @@ public class MRClientService extends AbstractService
       TaskId taskId = request.getTaskId();
       GetTaskReportResponse response = 
         recordFactory.newRecordInstance(GetTaskReportResponse.class);
-      response.setTaskReport(
-          verifyAndGetTask(taskId, JobACL.VIEW_JOB).getReport());
+      response.setTaskReport(verifyAndGetTask(taskId, false).getReport());
       return response;
     }
 
@@ -266,7 +256,7 @@ public class MRClientService extends AbstractService
       JobId jobId = request.getJobId();
       int fromEventId = request.getFromEventId();
       int maxEvents = request.getMaxEvents();
-      Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
+      Job job = verifyAndGetJob(jobId, false);
       
       GetTaskAttemptCompletionEventsResponse response = 
         recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsResponse.class);
@@ -280,11 +270,9 @@ public class MRClientService extends AbstractService
     public KillJobResponse killJob(KillJobRequest request) 
       throws IOException {
       JobId jobId = request.getJobId();
-      UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
-      String message = "Kill job " + jobId + " received from " + callerUGI
-          + " at " + Server.getRemoteAddress();
+      String message = "Kill Job received from client " + jobId;
       LOG.info(message);
-      verifyAndGetJob(jobId, JobACL.MODIFY_JOB);
+  	  verifyAndGetJob(jobId, true);
       appContext.getEventHandler().handle(
           new JobDiagnosticsUpdateEvent(jobId, message));
       appContext.getEventHandler().handle(
@@ -299,11 +287,9 @@ public class MRClientService extends AbstractService
     public KillTaskResponse killTask(KillTaskRequest request) 
       throws IOException {
       TaskId taskId = request.getTaskId();
-      UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
-      String message = "Kill task " + taskId + " received from " + callerUGI
-          + " at " + Server.getRemoteAddress();
+      String message = "Kill task received from client " + taskId;
       LOG.info(message);
-      verifyAndGetTask(taskId, JobACL.MODIFY_JOB);
+      verifyAndGetTask(taskId, true);
       appContext.getEventHandler().handle(
           new TaskEvent(taskId, TaskEventType.T_KILL));
       KillTaskResponse response = 
@@ -316,12 +302,9 @@ public class MRClientService extends AbstractService
     public KillTaskAttemptResponse killTaskAttempt(
         KillTaskAttemptRequest request) throws IOException {
       TaskAttemptId taskAttemptId = request.getTaskAttemptId();
-      UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
-      String message = "Kill task attempt " + taskAttemptId
-          + " received from " + callerUGI + " at "
-          + Server.getRemoteAddress();
+      String message = "Kill task attempt received from client " + taskAttemptId;
       LOG.info(message);
-      verifyAndGetAttempt(taskAttemptId, JobACL.MODIFY_JOB);
+      verifyAndGetAttempt(taskAttemptId, true);
       appContext.getEventHandler().handle(
           new TaskAttemptDiagnosticsUpdateEvent(taskAttemptId, message));
       appContext.getEventHandler().handle(
@@ -339,8 +322,8 @@ public class MRClientService extends AbstractService
       
       GetDiagnosticsResponse response = 
         recordFactory.newRecordInstance(GetDiagnosticsResponse.class);
-      response.addAllDiagnostics(verifyAndGetAttempt(taskAttemptId,
-          JobACL.VIEW_JOB).getDiagnostics());
+      response.addAllDiagnostics(
+          verifyAndGetAttempt(taskAttemptId, false).getDiagnostics());
       return response;
     }
 
@@ -349,12 +332,9 @@ public class MRClientService extends AbstractService
     public FailTaskAttemptResponse failTaskAttempt(
         FailTaskAttemptRequest request) throws IOException {
       TaskAttemptId taskAttemptId = request.getTaskAttemptId();
-      UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
-      String message = "Fail task attempt " + taskAttemptId
-          + " received from " + callerUGI + " at "
-          + Server.getRemoteAddress();
+      String message = "Fail task attempt received from client " + taskAttemptId;
       LOG.info(message);
-      verifyAndGetAttempt(taskAttemptId, JobACL.MODIFY_JOB);
+      verifyAndGetAttempt(taskAttemptId, true);
       appContext.getEventHandler().handle(
           new TaskAttemptDiagnosticsUpdateEvent(taskAttemptId, message));
       appContext.getEventHandler().handle(
@@ -376,7 +356,7 @@ public class MRClientService extends AbstractService
       GetTaskReportsResponse response = 
         recordFactory.newRecordInstance(GetTaskReportsResponse.class);
       
-      Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
+      Job job = verifyAndGetJob(jobId, false);
       Collection<Task> tasks = job.getTasks(taskType).values();
       LOG.info("Getting task report for " + taskType + "   " + jobId
           + ". Report-size will be " + tasks.size());

+ 0 - 85
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java

@@ -18,20 +18,13 @@
 
 package org.apache.hadoop.mapreduce.v2.app;
 
-import static org.junit.Assert.fail;
-
-import java.security.PrivilegedExceptionAction;
 import java.util.Iterator;
 import java.util.List;
 
 import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.JobACL;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
-import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetCountersRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDiagnosticsRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportRequest;
@@ -39,9 +32,6 @@ import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptCompleti
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptReportRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportsRequest;
-import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillJobRequest;
-import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskAttemptRequest;
-import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskRequest;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
@@ -61,8 +51,6 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -181,79 +169,6 @@ public class TestMRClientService {
     app.waitForState(job, JobState.SUCCEEDED);
   }
 
-  @Test
-  public void testViewAclOnlyCannotModify() throws Exception {
-    final MRAppWithClientService app = new MRAppWithClientService(1, 0, false);
-    final Configuration conf = new Configuration();
-    conf.setBoolean(MRConfig.MR_ACLS_ENABLED, true);
-    conf.set(MRJobConfig.JOB_ACL_VIEW_JOB, "viewonlyuser");
-    Job job = app.submit(conf);
-    app.waitForState(job, JobState.RUNNING);
-    Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
-    Iterator<Task> it = job.getTasks().values().iterator();
-    Task task = it.next();
-    app.waitForState(task, TaskState.RUNNING);
-    TaskAttempt attempt = task.getAttempts().values().iterator().next();
-    app.waitForState(attempt, TaskAttemptState.RUNNING);
-
-    UserGroupInformation viewOnlyUser =
-        UserGroupInformation.createUserForTesting(
-            "viewonlyuser", new String[] {});
-    Assert.assertTrue("viewonlyuser cannot view job",
-        job.checkAccess(viewOnlyUser, JobACL.VIEW_JOB));
-    Assert.assertFalse("viewonlyuser can modify job",
-        job.checkAccess(viewOnlyUser, JobACL.MODIFY_JOB));
-    MRClientProtocol client = viewOnlyUser.doAs(
-        new PrivilegedExceptionAction<MRClientProtocol>() {
-          @Override
-          public MRClientProtocol run() throws Exception {
-            YarnRPC rpc = YarnRPC.create(conf);
-            return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
-                app.clientService.getBindAddress(), conf);
-          }
-        });
-
-    KillJobRequest killJobRequest = recordFactory.newRecordInstance(
-        KillJobRequest.class);
-    killJobRequest.setJobId(app.getJobId());
-    try {
-      client.killJob(killJobRequest);
-      fail("viewonlyuser killed job");
-    } catch (AccessControlException e) {
-      // pass
-    }
-
-    KillTaskRequest killTaskRequest = recordFactory.newRecordInstance(
-        KillTaskRequest.class);
-    killTaskRequest.setTaskId(task.getID());
-    try {
-      client.killTask(killTaskRequest);
-      fail("viewonlyuser killed task");
-    } catch (AccessControlException e) {
-      // pass
-    }
-
-    KillTaskAttemptRequest killTaskAttemptRequest =
-        recordFactory.newRecordInstance(KillTaskAttemptRequest.class);
-    killTaskAttemptRequest.setTaskAttemptId(attempt.getID());
-    try {
-      client.killTaskAttempt(killTaskAttemptRequest);
-      fail("viewonlyuser killed task attempt");
-    } catch (AccessControlException e) {
-      // pass
-    }
-
-    FailTaskAttemptRequest failTaskAttemptRequest =
-        recordFactory.newRecordInstance(FailTaskAttemptRequest.class);
-    failTaskAttemptRequest.setTaskAttemptId(attempt.getID());
-    try {
-      client.failTaskAttempt(failTaskAttemptRequest);
-      fail("viewonlyuser killed task attempt");
-    } catch (AccessControlException e) {
-      // pass
-    }
-  }
-
   private void verifyJobReport(JobReport jr) {
     Assert.assertNotNull("JobReport is null", jr);
     List<AMInfo> amInfos = jr.getAMInfos();

+ 0 - 2
hadoop-yarn-project/CHANGES.txt

@@ -10,8 +10,6 @@ Release 2.1.1-beta - UNRELEASED
 
     YARN-589. Expose a REST API for monitoring the fair scheduler (Sandy Ryza).
     
-    YARN-707. Add user info in the YARN ClientToken (vinodkv via jlowe)
-
   OPTIMIZATIONS
 
   BUG FIXES

+ 3 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientToAMTokenIdentifier.java

@@ -39,7 +39,6 @@ public class ClientToAMTokenIdentifier extends TokenIdentifier {
   public static final Text KIND_NAME = new Text("YARN_CLIENT_TOKEN");
 
   private ApplicationAttemptId applicationAttemptId;
-  private Text applicationSubmitter = new Text();
 
   // TODO: Add more information in the tokenID such that it is not
   // transferrable, more secure etc.
@@ -47,27 +46,21 @@ public class ClientToAMTokenIdentifier extends TokenIdentifier {
   public ClientToAMTokenIdentifier() {
   }
 
-  public ClientToAMTokenIdentifier(ApplicationAttemptId id, String appSubmitter) {
+  public ClientToAMTokenIdentifier(ApplicationAttemptId id) {
     this();
     this.applicationAttemptId = id;
-    this.applicationSubmitter = new Text(appSubmitter);
   }
 
   public ApplicationAttemptId getApplicationAttemptID() {
     return this.applicationAttemptId;
   }
 
-  public String getApplicationSubmitter() {
-    return this.applicationSubmitter.toString();
-  }
-
   @Override
   public void write(DataOutput out) throws IOException {
     out.writeLong(this.applicationAttemptId.getApplicationId()
       .getClusterTimestamp());
     out.writeInt(this.applicationAttemptId.getApplicationId().getId());
     out.writeInt(this.applicationAttemptId.getAttemptId());
-    this.applicationSubmitter.write(out);
   }
 
   @Override
@@ -75,7 +68,6 @@ public class ClientToAMTokenIdentifier extends TokenIdentifier {
     this.applicationAttemptId =
         ApplicationAttemptId.newInstance(
           ApplicationId.newInstance(in.readLong(), in.readInt()), in.readInt());
-    this.applicationSubmitter.readFields(in);
   }
 
   @Override
@@ -85,11 +77,10 @@ public class ClientToAMTokenIdentifier extends TokenIdentifier {
 
   @Override
   public UserGroupInformation getUser() {
-    if (this.applicationSubmitter == null) {
+    if (this.applicationAttemptId == null) {
       return null;
     }
-    return UserGroupInformation.createRemoteUser(this.applicationSubmitter
-      .toString());
+    return UserGroupInformation.createRemoteUser(this.applicationAttemptId.toString());
   }
 
   @InterfaceAudience.Private

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -722,7 +722,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         // create clientToAMToken
         appAttempt.clientToAMToken =
             new Token<ClientToAMTokenIdentifier>(new ClientToAMTokenIdentifier(
-              appAttempt.applicationAttemptId, appAttempt.user),
+              appAttempt.applicationAttemptId),
               appAttempt.rmContext.getClientToAMTokenSecretManager());
       }
 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStore.java

@@ -367,7 +367,7 @@ public class TestRMStateStore {
     appToken.setService(new Text("appToken service"));
 
     ClientToAMTokenIdentifier clientToAMTokenId =
-        new ClientToAMTokenIdentifier(attemptId, "user");
+        new ClientToAMTokenIdentifier(attemptId);
     clientToAMTokenMgr.registerApplication(attemptId);
     Token<ClientToAMTokenIdentifier> clientToAMToken =
         new Token<ClientToAMTokenIdentifier>(clientToAMTokenId, clientToAMTokenMgr);

+ 17 - 47
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java

@@ -115,6 +115,7 @@ public class TestClientToAMTokens {
     private final byte[] secretKey;
     private InetSocketAddress address;
     private boolean pinged = false;
+    private ClientToAMTokenSecretManager secretManager;
     
     public CustomAM(ApplicationAttemptId appId, byte[] secretKey) {
       super("CustomAM");
@@ -131,14 +132,12 @@ public class TestClientToAMTokens {
     protected void serviceStart() throws Exception {
       Configuration conf = getConfig();
 
+      secretManager = new ClientToAMTokenSecretManager(this.appAttemptId, secretKey);
       Server server;
       try {
         server =
-            new RPC.Builder(conf)
-              .setProtocol(CustomProtocol.class)
-              .setNumHandlers(1)
-              .setSecretManager(
-                new ClientToAMTokenSecretManager(this.appAttemptId, secretKey))
+            new RPC.Builder(conf).setProtocol(CustomProtocol.class)
+              .setNumHandlers(1).setSecretManager(secretManager)
               .setInstance(this).build();
       } catch (Exception e) {
         throw new YarnRuntimeException(e);
@@ -147,10 +146,14 @@ public class TestClientToAMTokens {
       this.address = NetUtils.getConnectAddress(server);
       super.serviceStart();
     }
+    
+    public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
+      return this.secretManager;
+    }
   }
 
   @Test
-  public void testClientToAMTokenss() throws Exception {
+  public void testClientToAMs() throws Exception {
 
     final Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
@@ -201,7 +204,7 @@ public class TestClientToAMTokens {
     GetApplicationReportResponse reportResponse =
         rm.getClientRMService().getApplicationReport(request);
     ApplicationReport appReport = reportResponse.getApplicationReport();
-    org.apache.hadoop.yarn.api.records.Token originalClientToAMToken =
+    org.apache.hadoop.yarn.api.records.Token clientToAMToken =
         appReport.getClientToAMToken();
 
     ApplicationAttemptId appAttempt = app.getCurrentAppAttempt().getAppAttemptId();
@@ -256,47 +259,17 @@ public class TestClientToAMTokens {
       Assert.assertFalse(am.pinged);
     }
 
+    // Verify denial for a malicious user
+    UserGroupInformation ugi = UserGroupInformation.createRemoteUser("me");
     Token<ClientToAMTokenIdentifier> token =
-        ConverterUtils.convertFromYarn(originalClientToAMToken, am.address);
-
-    // Verify denial for a malicious user with tampered ID
-    verifyTokenWithTamperedID(conf, am, token);
-
-    // Verify denial for a malicious user with tampered user-name
-    verifyTokenWithTamperedUserName(conf, am, token);
+        ConverterUtils.convertFromYarn(clientToAMToken, am.address);
 
-    // Now for an authenticated user
-    verifyValidToken(conf, am, token);
-  }
-
-  private void verifyTokenWithTamperedID(final Configuration conf,
-      final CustomAM am, Token<ClientToAMTokenIdentifier> token)
-      throws IOException {
     // Malicious user, messes with appId
-    UserGroupInformation ugi = UserGroupInformation.createRemoteUser("me");
     ClientToAMTokenIdentifier maliciousID =
         new ClientToAMTokenIdentifier(BuilderUtils.newApplicationAttemptId(
-          BuilderUtils.newApplicationId(am.appAttemptId.getApplicationId()
-            .getClusterTimestamp(), 42), 43), UserGroupInformation
-          .getCurrentUser().getShortUserName());
+          BuilderUtils.newApplicationId(app.getApplicationId()
+            .getClusterTimestamp(), 42), 43));
 
-    verifyTamperedToken(conf, am, token, ugi, maliciousID);
-  }
-
-  private void verifyTokenWithTamperedUserName(final Configuration conf,
-      final CustomAM am, Token<ClientToAMTokenIdentifier> token)
-      throws IOException {
-    // Malicious user, messes with appId
-    UserGroupInformation ugi = UserGroupInformation.createRemoteUser("me");
-    ClientToAMTokenIdentifier maliciousID =
-        new ClientToAMTokenIdentifier(am.appAttemptId, "evilOrc");
-
-    verifyTamperedToken(conf, am, token, ugi, maliciousID);
-  }
-  
-  private void verifyTamperedToken(final Configuration conf, final CustomAM am,
-      Token<ClientToAMTokenIdentifier> token, UserGroupInformation ugi,
-      ClientToAMTokenIdentifier maliciousID) {
     Token<ClientToAMTokenIdentifier> maliciousToken =
         new Token<ClientToAMTokenIdentifier>(maliciousID.getBytes(),
           token.getPassword(), token.getKind(),
@@ -336,12 +309,8 @@ public class TestClientToAMTokens {
               + "Mismatched response."));
       Assert.assertFalse(am.pinged);
     }
-  }
 
-  private void verifyValidToken(final Configuration conf, final CustomAM am,
-      Token<ClientToAMTokenIdentifier> token) throws IOException,
-      InterruptedException {
-    UserGroupInformation ugi;
+    // Now for an authenticated user
     ugi = UserGroupInformation.createRemoteUser("me");
     ugi.addToken(token);
 
@@ -357,4 +326,5 @@ public class TestClientToAMTokens {
       }
     });
   }
+
 }