瀏覽代碼

YARN-717. Put object creation factories for Token in the class itself and remove useless derivations for specific tokens. Contributed by Jian He.
MAPREDUCE-5289. Updated MR App to use Token directly after YARN-717. Contributed by Jian He.
svn merge --ignore-ancestry -c 1488616 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1488617 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 12 年之前
父節點
當前提交
41ed12124c
共有 77 個文件被更改,包括 365 次插入510 次删除
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherEvent.java
  3. 5 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
  4. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
  5. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
  6. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
  7. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
  8. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/CancelDelegationTokenRequest.java
  9. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/GetDelegationTokenResponse.java
  10. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/RenewDelegationTokenRequest.java
  11. 9 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/impl/pb/CancelDelegationTokenRequestPBImpl.java
  12. 9 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/impl/pb/GetDelegationTokenResponsePBImpl.java
  13. 9 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/impl/pb/RenewDelegationTokenRequestPBImpl.java
  14. 2 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/MRDelegationTokenRenewer.java
  15. 3 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java
  16. 1 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
  17. 1 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  18. 3 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
  19. 2 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java
  20. 9 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java
  21. 3 0
      hadoop-yarn-project/CHANGES.txt
  22. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java
  23. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/CancelDelegationTokenRequest.java
  24. 10 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetDelegationTokenResponse.java
  25. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RenewDelegationTokenRequest.java
  26. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
  27. 9 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/CancelDelegationTokenRequestPBImpl.java
  28. 9 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetDelegationTokenResponsePBImpl.java
  29. 9 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RenewDelegationTokenRequestPBImpl.java
  30. 9 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainerRequestPBImpl.java
  31. 14 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
  32. 0 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ClientToken.java
  33. 18 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
  34. 0 43
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerToken.java
  35. 0 32
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/DelegationToken.java
  36. 22 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Token.java
  37. 9 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
  38. 0 33
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ClientTokenPBImpl.java
  39. 9 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java
  40. 0 33
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerTokenPBImpl.java
  41. 0 34
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/DelegationTokenPBImpl.java
  42. 27 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/TokenPBImpl.java
  43. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClient.java
  44. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java
  45. 9 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java
  46. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java
  47. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java
  48. 9 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java
  49. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java
  50. 11 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java
  51. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
  52. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
  53. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  54. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
  55. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java
  56. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
  57. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
  58. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
  59. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
  60. 13 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
  61. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
  62. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
  63. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  64. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  65. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  66. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java
  67. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  68. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
  69. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
  70. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
  71. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  72. 12 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
  73. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
  74. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
  75. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java
  76. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java
  77. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

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

@@ -130,6 +130,9 @@ Release 2.1.0-beta - UNRELEASED
     ClusterMetrics for binary compatibility with 1.x APIs. (Mayank Bansal via
     vinodkv)
 
+    MAPREDUCE-5289. Updated MR App to use Token directly after YARN-717. (Jian He
+    via vinodkv)
+
   OPTIMIZATIONS
 
     MAPREDUCE-4974. Optimising the LineRecordReader initialize() method 

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherEvent.java

@@ -18,10 +18,10 @@
 
 package org.apache.hadoop.mapreduce.v2.app.launcher;
 
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 
 public class ContainerLauncherEvent 
     extends AbstractEvent<ContainerLauncher.EventType> {
@@ -29,12 +29,12 @@ public class ContainerLauncherEvent
   private TaskAttemptId taskAttemptID;
   private ContainerId containerID;
   private String containerMgrAddress;
-  private ContainerToken containerToken;
+  private Token containerToken;
 
   public ContainerLauncherEvent(TaskAttemptId taskAttemptID, 
       ContainerId containerID,
       String containerMgrAddress,
-      ContainerToken containerToken,
+      Token containerToken,
       ContainerLauncher.EventType type) {
     super(type);
     this.taskAttemptID = taskAttemptID;
@@ -55,7 +55,7 @@ public class ContainerLauncherEvent
     return containerMgrAddress;
   }
 
-  public ContainerToken getContainerToken() {
+  public Token getContainerToken() {
     return containerToken;
   }
 

+ 5 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java

@@ -55,7 +55,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.service.AbstractService;
@@ -115,10 +114,11 @@ public class ContainerLauncherImpl extends AbstractService implements
     private TaskAttemptId taskAttemptID;
     private ContainerId containerID;
     final private String containerMgrAddress;
-    private ContainerToken containerToken;
+    private org.apache.hadoop.yarn.api.records.Token containerToken;
     
     public Container(TaskAttemptId taId, ContainerId containerID,
-        String containerMgrAddress, ContainerToken containerToken) {
+        String containerMgrAddress,
+        org.apache.hadoop.yarn.api.records.Token containerToken) {
       this.state = ContainerState.PREP;
       this.taskAttemptID = taId;
       this.containerMgrAddress = containerMgrAddress;
@@ -345,7 +345,8 @@ public class ContainerLauncherImpl extends AbstractService implements
   }
 
   protected ContainerManager getCMProxy(ContainerId containerID,
-      final String containerManagerBindAddr, ContainerToken containerToken)
+      final String containerManagerBindAddr,
+      org.apache.hadoop.yarn.api.records.Token containerToken)
       throws IOException {
 
     final InetSocketAddress cmAddr =

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java

@@ -90,7 +90,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -516,7 +516,7 @@ public class MRApp extends MRAppMaster {
         ContainerTokenIdentifier containerTokenIdentifier =
             new ContainerTokenIdentifier(cId, nodeId.toString(), "user",
               resource, System.currentTimeMillis() + 10000, 42, 42);
-        ContainerToken containerToken =
+        Token containerToken =
             BuilderUtils.newContainerToken(nodeId, "password".getBytes(),
               containerTokenIdentifier);
         Container container = Container.newInstance(cId, nodeId,

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java

@@ -46,7 +46,7 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.ContainerManager;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.junit.Test;
 
 /**
@@ -226,7 +226,7 @@ public class TestFail {
 
         @Override
         protected ContainerManager getCMProxy(ContainerId contianerID,
-            String containerManagerBindAddr, ContainerToken containerToken)
+            String containerManagerBindAddr, Token containerToken)
             throws IOException {
           try {
             synchronized (this) {

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java

@@ -61,7 +61,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -347,7 +347,7 @@ public class TestContainerLauncher {
       return new ContainerLauncherImpl(context) {
         @Override
         protected ContainerManager getCMProxy(ContainerId containerID,
-            String containerManagerBindAddr, ContainerToken containerToken)
+            String containerManagerBindAddr, Token containerToken)
             throws IOException {
           // make proxy connect to our local containerManager server
           ContainerManager proxy = (ContainerManager) rpc.getProxy(

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java

@@ -55,9 +55,9 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
 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.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
@@ -407,7 +407,7 @@ public class TestContainerLauncherImpl {
     }
   }
   
-  private ContainerToken createNewContainerToken(ContainerId contId,
+  private Token createNewContainerToken(ContainerId contId,
       String containerManagerAddr) {
     long currentTime = System.currentTimeMillis();
     return BuilderUtils.newContainerToken(NodeId.newInstance("127.0.0.1",

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/CancelDelegationTokenRequest.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.mapreduce.v2.api.protocolrecords;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
+import org.apache.hadoop.yarn.api.records.Token;
 
 /**
  * The request issued by the client to the {@code ResourceManager} to cancel a
@@ -29,6 +29,6 @@ import org.apache.hadoop.yarn.api.records.DelegationToken;
 @Public
 @Evolving
 public interface CancelDelegationTokenRequest {
-  DelegationToken getDelegationToken();
-  void setDelegationToken(DelegationToken dToken);
+  Token getDelegationToken();
+  void setDelegationToken(Token dToken);
 }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/GetDelegationTokenResponse.java

@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.mapreduce.v2.api.protocolrecords;
 
-import org.apache.hadoop.yarn.api.records.DelegationToken;
+import org.apache.hadoop.yarn.api.records.Token;
 
 public interface GetDelegationTokenResponse {
-  void setDelegationToken(DelegationToken clientDToken);
-  DelegationToken getDelegationToken();
+  void setDelegationToken(Token clientDToken);
+  Token getDelegationToken();
 }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/RenewDelegationTokenRequest.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.mapreduce.v2.api.protocolrecords;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
+import org.apache.hadoop.yarn.api.records.Token;
 
 /**
  * The request issued by the client to renew a delegation token from
@@ -29,6 +29,6 @@ import org.apache.hadoop.yarn.api.records.DelegationToken;
 @Public
 @Evolving
 public interface RenewDelegationTokenRequest {
-  DelegationToken getDelegationToken();
-  void setDelegationToken(DelegationToken dToken);
+  Token getDelegationToken();
+  void setDelegationToken(Token dToken);
 }

+ 9 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/impl/pb/CancelDelegationTokenRequestPBImpl.java

@@ -21,9 +21,9 @@ import org.apache.hadoop.mapreduce.v2.api.protocolrecords.CancelDelegationTokenR
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProtoOrBuilder;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.api.records.impl.pb.DelegationTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 
 public class CancelDelegationTokenRequestPBImpl extends
     ProtoBase<CancelDelegationTokenRequestProto> implements
@@ -44,10 +44,10 @@ public class CancelDelegationTokenRequestPBImpl extends
     this.viaProto = true;
   }
     
-  DelegationToken token;
+  Token token;
 
   @Override
-  public DelegationToken getDelegationToken() {
+  public Token getDelegationToken() {
     CancelDelegationTokenRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (this.token != null) {
       return this.token;
@@ -57,7 +57,7 @@ public class CancelDelegationTokenRequestPBImpl extends
   }
 
   @Override
-  public void setDelegationToken(DelegationToken token) {
+  public void setDelegationToken(Token token) {
     maybeInitBuilder();
     if (token == null) 
       builder.clearToken();
@@ -95,11 +95,11 @@ public class CancelDelegationTokenRequestPBImpl extends
   }
 
 
-  private DelegationTokenPBImpl convertFromProtoFormat(TokenProto p) {
-    return new DelegationTokenPBImpl(p);
+  private TokenPBImpl convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
   }
 
-  private TokenProto convertToProtoFormat(DelegationToken t) {
-    return ((DelegationTokenPBImpl)t).getProto();
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl)t).getProto();
   }
 }

+ 9 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/impl/pb/GetDelegationTokenResponsePBImpl.java

@@ -21,14 +21,14 @@ import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenResp
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProtoOrBuilder;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.api.records.impl.pb.DelegationTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 
 public class GetDelegationTokenResponsePBImpl extends
       ProtoBase<GetDelegationTokenResponseProto> implements GetDelegationTokenResponse {
   
-  DelegationToken mrToken;
+  Token mrToken;
   
 
   GetDelegationTokenResponseProto proto = 
@@ -47,7 +47,7 @@ public class GetDelegationTokenResponsePBImpl extends
   }
   
   @Override
-  public DelegationToken getDelegationToken() {
+  public Token getDelegationToken() {
     GetDelegationTokenResponseProtoOrBuilder p = viaProto ? proto : builder;
     if (this.mrToken != null) {
       return this.mrToken;
@@ -60,7 +60,7 @@ public class GetDelegationTokenResponsePBImpl extends
   }
   
   @Override
-  public void setDelegationToken(DelegationToken mrToken) {
+  public void setDelegationToken(Token mrToken) {
     maybeInitBuilder();
     if (mrToken == null) 
       builder.getToken();
@@ -97,11 +97,11 @@ public class GetDelegationTokenResponsePBImpl extends
     viaProto = false;
   }
 
-  private DelegationTokenPBImpl convertFromProtoFormat(TokenProto p) {
-    return new DelegationTokenPBImpl(p);
+  private TokenPBImpl convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
   }
 
-  private TokenProto convertToProtoFormat(DelegationToken t) {
-    return ((DelegationTokenPBImpl)t).getProto();
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl)t).getProto();
   }
 }

+ 9 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/protocolrecords/impl/pb/RenewDelegationTokenRequestPBImpl.java

@@ -21,9 +21,9 @@ import org.apache.hadoop.mapreduce.v2.api.protocolrecords.RenewDelegationTokenRe
 import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProtoOrBuilder;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.api.records.impl.pb.DelegationTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 
 public class RenewDelegationTokenRequestPBImpl extends
     ProtoBase<RenewDelegationTokenRequestProto> implements
@@ -44,10 +44,10 @@ public class RenewDelegationTokenRequestPBImpl extends
     this.viaProto = true;
   }
 
-  DelegationToken token;
+  Token token;
 
   @Override
-  public DelegationToken getDelegationToken() {
+  public Token getDelegationToken() {
     RenewDelegationTokenRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (this.token != null) {
       return this.token;
@@ -57,7 +57,7 @@ public class RenewDelegationTokenRequestPBImpl extends
   }
 
   @Override
-  public void setDelegationToken(DelegationToken token) {
+  public void setDelegationToken(Token token) {
     maybeInitBuilder();
     if (token == null)
       builder.clearToken();
@@ -93,11 +93,11 @@ public class RenewDelegationTokenRequestPBImpl extends
     viaProto = false;
   }
 
-  private DelegationTokenPBImpl convertFromProtoFormat(TokenProto p) {
-    return new DelegationTokenPBImpl(p);
+  private TokenPBImpl convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
   }
 
-  private TokenProto convertToProtoFormat(DelegationToken t) {
-    return ((DelegationTokenPBImpl) t).getProto();
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl) t).getProto();
   }
 }

+ 2 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/security/MRDelegationTokenRenewer.java

@@ -36,7 +36,6 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
@@ -56,7 +55,7 @@ public class MRDelegationTokenRenewer extends TokenRenewer {
   public long renew(Token<?> token, Configuration conf) throws IOException,
       InterruptedException {
 
-    DelegationToken dToken = BuilderUtils.newDelegationToken(
+    org.apache.hadoop.yarn.api.records.Token dToken = BuilderUtils.newDelegationToken(
         token.getIdentifier(), token.getKind().toString(), token.getPassword(),
         token.getService().toString());
 
@@ -77,7 +76,7 @@ public class MRDelegationTokenRenewer extends TokenRenewer {
   public void cancel(Token<?> token, Configuration conf) throws IOException,
       InterruptedException {
 
-    DelegationToken dToken = BuilderUtils.newDelegationToken(
+    org.apache.hadoop.yarn.api.records.Token dToken = BuilderUtils.newDelegationToken(
         token.getIdentifier(), token.getKind().toString(), token.getPassword(),
         token.getService().toString());
 

+ 3 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java

@@ -79,7 +79,6 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -344,7 +343,7 @@ public class HistoryClientService extends AbstractService {
       Token<MRDelegationTokenIdentifier> realJHSToken =
           new Token<MRDelegationTokenIdentifier>(tokenIdentifier,
               jhsDTSecretManager);
-      DelegationToken mrDToken = BuilderUtils.newDelegationToken(
+      org.apache.hadoop.yarn.api.records.Token mrDToken = BuilderUtils.newDelegationToken(
         realJHSToken.getIdentifier(), realJHSToken.getKind().toString(),
         realJHSToken.getPassword(), realJHSToken.getService().toString());
       response.setDelegationToken(mrDToken);
@@ -359,7 +358,7 @@ public class HistoryClientService extends AbstractService {
               "Delegation Token can be renewed only with kerberos authentication");
         }
 
-        DelegationToken protoToken = request.getDelegationToken();
+        org.apache.hadoop.yarn.api.records.Token protoToken = request.getDelegationToken();
         Token<MRDelegationTokenIdentifier> token =
             new Token<MRDelegationTokenIdentifier>(
                 protoToken.getIdentifier().array(), protoToken.getPassword()
@@ -382,7 +381,7 @@ public class HistoryClientService extends AbstractService {
               "Delegation Token can be cancelled only with kerberos authentication");
         }
 
-        DelegationToken protoToken = request.getDelegationToken();
+        org.apache.hadoop.yarn.api.records.Token protoToken = request.getDelegationToken();
         Token<MRDelegationTokenIdentifier> token =
             new Token<MRDelegationTokenIdentifier>(
                 protoToken.getIdentifier().array(), protoToken.getPassword()

+ 1 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

@@ -33,7 +33,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.http.HttpConfig;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -64,13 +63,11 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptReport;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ClientToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
@@ -78,7 +75,6 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.client.ClientTokenIdentifier;
-import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.ProtoUtils;
 
 public class ClientServiceDelegate {
@@ -184,7 +180,7 @@ public class ClientServiceDelegate {
           serviceAddr = NetUtils.createSocketAddrForHost(
               application.getHost(), application.getRpcPort());
           if (UserGroupInformation.isSecurityEnabled()) {
-            ClientToken clientToken = application.getClientToken();
+            org.apache.hadoop.yarn.api.records.Token clientToken = application.getClientToken();
             Token<ClientTokenIdentifier> token =
                 ProtoUtils.convertFromProtoFormat(clientToken, serviceAddr);
             newUgi.addToken(token);

+ 1 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java

@@ -72,7 +72,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -207,7 +206,7 @@ public class YARNRunner implements ClientProtocol {
     GetDelegationTokenRequest request = recordFactory
       .newRecordInstance(GetDelegationTokenRequest.class);
     request.setRenewer(Master.getMasterPrincipal(conf));
-    DelegationToken mrDelegationToken;
+    org.apache.hadoop.yarn.api.records.Token mrDelegationToken;
     mrDelegationToken = hsProxy.getDelegationToken(request)
         .getDelegationToken();
     return ProtoUtils.convertFromProtoFormat(mrDelegationToken,

+ 3 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java

@@ -79,7 +79,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
@@ -283,7 +282,7 @@ public class TestYARNRunner extends TestCase {
       token.setKind(RMDelegationTokenIdentifier.KIND_NAME);
 
       // Setup mock history token
-      DelegationToken historyToken = BuilderUtils.newDelegationToken(
+      org.apache.hadoop.yarn.api.records.Token historyToken = BuilderUtils.newDelegationToken(
           new byte[0], MRDelegationTokenIdentifier.KIND_NAME.toString(),
           new byte[0], hsTokenSevice.toString());
       GetDelegationTokenResponse getDtResponse = Records
@@ -359,8 +358,8 @@ public class TestYARNRunner extends TestCase {
             // check that the renewer matches the cluster's RM principal
             assertEquals(masterPrincipal, request.getRenewer() );
 
-            DelegationToken token =
-                recordFactory.newRecordInstance(DelegationToken.class);
+            org.apache.hadoop.yarn.api.records.Token token =
+                recordFactory.newRecordInstance(org.apache.hadoop.yarn.api.records.Token.class);
             // none of these fields matter for the sake of the test
             token.setKind("");
             token.setService("");

+ 2 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -98,8 +97,8 @@ public class TestYarnClientProtocolProvider extends TestCase {
       YARNRunner yrunner = (YARNRunner) cluster.getClient();
       GetDelegationTokenResponse getDTResponse = 
           recordFactory.newRecordInstance(GetDelegationTokenResponse.class);
-      DelegationToken rmDTToken = recordFactory.newRecordInstance(
-          DelegationToken.class);
+      org.apache.hadoop.yarn.api.records.Token rmDTToken = recordFactory.newRecordInstance(
+        org.apache.hadoop.yarn.api.records.Token.class);
       rmDTToken.setIdentifier(ByteBuffer.wrap(new byte[2]));
       rmDTToken.setKind("Testclusterkind");
       rmDTToken.setPassword(ByteBuffer.wrap("testcluster".getBytes()));

+ 9 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java

@@ -45,7 +45,7 @@ import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.util.ProtoUtils;
@@ -107,7 +107,7 @@ public class TestJHSSecurity {
       loggedInUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
 
 
-      DelegationToken token = getDelegationToken(loggedInUser, hsService,
+      Token token = getDelegationToken(loggedInUser, hsService,
           loggedInUser.getShortUserName());
       tokenFetchTime = System.currentTimeMillis();
       LOG.info("Got delegation token at: " + tokenFetchTime);
@@ -207,16 +207,16 @@ public class TestJHSSecurity {
     }
   }
 
-  private DelegationToken getDelegationToken(
+  private Token getDelegationToken(
       final UserGroupInformation loggedInUser,
       final MRClientProtocol hsService, final String renewerString)
       throws IOException, InterruptedException {
     // Get the delegation token directly as it is a little difficult to setup
     // the kerberos based rpc.
-    DelegationToken token = loggedInUser
-        .doAs(new PrivilegedExceptionAction<DelegationToken>() {
+    Token token = loggedInUser
+        .doAs(new PrivilegedExceptionAction<Token>() {
           @Override
-          public DelegationToken run() throws IOException {
+          public Token run() throws IOException {
             GetDelegationTokenRequest request = Records
                 .newRecord(GetDelegationTokenRequest.class);
             request.setRenewer(renewerString);
@@ -228,7 +228,7 @@ public class TestJHSSecurity {
   }
 
   private long renewDelegationToken(final UserGroupInformation loggedInUser,
-      final MRClientProtocol hsService, final DelegationToken dToken)
+      final MRClientProtocol hsService, final Token dToken)
       throws IOException, InterruptedException {
     long nextExpTime = loggedInUser.doAs(new PrivilegedExceptionAction<Long>() {
 
@@ -244,7 +244,7 @@ public class TestJHSSecurity {
   }
 
   private void cancelDelegationToken(final UserGroupInformation loggedInUser,
-      final MRClientProtocol hsService, final DelegationToken dToken)
+      final MRClientProtocol hsService, final Token dToken)
       throws IOException, InterruptedException {
 
     loggedInUser.doAs(new PrivilegedExceptionAction<Void>() {
@@ -259,7 +259,7 @@ public class TestJHSSecurity {
     });
   }
 
-  private MRClientProtocol getMRClientProtocol(DelegationToken token,
+  private MRClientProtocol getMRClientProtocol(Token token,
       final InetSocketAddress hsAddress, String user, final Configuration conf) {
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
     ugi.addToken(ProtoUtils.convertFromProtoFormat(token, hsAddress));

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

@@ -221,6 +221,9 @@ Release 2.1.0-beta - UNRELEASED
 
     YARN-660. Improve AMRMClient with matching requests (bikas)
 
+    YARN-717. Put object creation factories for Token in the class itself and
+    remove useless derivations for specific tokens. (Jian He via vinodkv)
+
   OPTIMIZATIONS
 
     YARN-512. Log aggregation root directory check is more expensive than it

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ClientRMProtocol.java

@@ -50,9 +50,9 @@ import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 
@@ -273,7 +273,7 @@ public interface ClientRMProtocol {
    * containers to be able to talk to the service using those tokens.
    * 
    *  <p> The <code>ResourceManager</code> responds with the delegation token
-   *  {@link DelegationToken} that can be used by the client to speak to this
+   *  {@link Token} that can be used by the client to speak to this
    *  service.
    * @param request request to get a delegation token for the client.
    * @return delegation token that can be used to talk to this service

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/CancelDelegationTokenRequest.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
+import org.apache.hadoop.yarn.api.records.Token;
 
 /**
  * The request issued by the client to the {@code ResourceManager} to cancel a
@@ -29,6 +29,6 @@ import org.apache.hadoop.yarn.api.records.DelegationToken;
 @Public
 @Evolving
 public interface CancelDelegationTokenRequest {
-  DelegationToken getDelegationToken();
-  void setDelegationToken(DelegationToken dToken);
+  Token getDelegationToken();
+  void setDelegationToken(Token dToken);
 }

+ 10 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetDelegationTokenResponse.java

@@ -20,7 +20,8 @@ package org.apache.hadoop.yarn.api.protocolrecords;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.api.records.Token;
 
 
 /**
@@ -32,6 +33,12 @@ import org.apache.hadoop.yarn.api.records.DelegationToken;
 @Public
 @Evolving
 public interface GetDelegationTokenResponse {
-  DelegationToken getRMDelegationToken();
-  void setRMDelegationToken(DelegationToken rmDTToken);
+
+  /**
+   * The Delegation tokens have a identifier which maps to
+   * {@link AbstractDelegationTokenIdentifier}.
+   *
+   */
+  Token getRMDelegationToken();
+  void setRMDelegationToken(Token rmDTToken);
 }

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RenewDelegationTokenRequest.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
+import org.apache.hadoop.yarn.api.records.Token;
 
 /**
  * The request issued by the client to renew a delegation token from
@@ -29,6 +29,6 @@ import org.apache.hadoop.yarn.api.records.DelegationToken;
 @Public
 @Evolving
 public interface RenewDelegationTokenRequest {
-  DelegationToken getDelegationToken();
-  void setDelegationToken(DelegationToken dToken);
+  Token getDelegationToken();
+  void setDelegationToken(Token dToken);
 }

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java

@@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ContainerManager;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 
 /**
  * <p>The request sent by the <code>ApplicationMaster</code> to the
@@ -62,9 +62,9 @@ public interface StartContainerRequest {
 
   @Public
   @Stable
-  public ContainerToken getContainerToken();
+  public Token getContainerToken();
 
   @Public
   @Stable
-  public void setContainerToken(ContainerToken container);
+  public void setContainerToken(Token container);
 }

+ 9 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/CancelDelegationTokenRequestPBImpl.java

@@ -21,9 +21,9 @@ import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequ
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProtoOrBuilder;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.api.records.impl.pb.DelegationTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 
 public class CancelDelegationTokenRequestPBImpl extends
     ProtoBase<CancelDelegationTokenRequestProto> implements
@@ -44,10 +44,10 @@ public class CancelDelegationTokenRequestPBImpl extends
     viaProto = true;
   }
 
-  DelegationToken token;
+  Token token;
 
   @Override
-  public DelegationToken getDelegationToken() {
+  public Token getDelegationToken() {
     CancelDelegationTokenRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (this.token != null) {
       return this.token;
@@ -57,7 +57,7 @@ public class CancelDelegationTokenRequestPBImpl extends
   }
 
   @Override
-  public void setDelegationToken(DelegationToken token) {
+  public void setDelegationToken(Token token) {
     maybeInitBuilder();
     if (token == null)
       builder.clearToken();
@@ -93,11 +93,11 @@ public class CancelDelegationTokenRequestPBImpl extends
     viaProto = false;
   }
 
-  private DelegationTokenPBImpl convertFromProtoFormat(TokenProto p) {
-    return new DelegationTokenPBImpl(p);
+  private TokenPBImpl convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
   }
 
-  private TokenProto convertToProtoFormat(DelegationToken t) {
-    return ((DelegationTokenPBImpl) t).getProto();
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl) t).getProto();
   }
 }

+ 9 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetDelegationTokenResponsePBImpl.java

@@ -22,14 +22,14 @@ import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRespons
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProtoOrBuilder;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.api.records.impl.pb.DelegationTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 
 public class GetDelegationTokenResponsePBImpl extends
 ProtoBase<GetDelegationTokenResponseProto> implements GetDelegationTokenResponse {
 
-  DelegationToken appToken;
+  Token appToken;
 
 
   GetDelegationTokenResponseProto proto = 
@@ -48,7 +48,7 @@ ProtoBase<GetDelegationTokenResponseProto> implements GetDelegationTokenResponse
   }
 
   @Override
-  public DelegationToken getRMDelegationToken() {
+  public Token getRMDelegationToken() {
     GetDelegationTokenResponseProtoOrBuilder p = viaProto ? proto : builder;
     if (this.appToken != null) {
       return this.appToken;
@@ -61,7 +61,7 @@ ProtoBase<GetDelegationTokenResponseProto> implements GetDelegationTokenResponse
   }
 
   @Override
-  public void setRMDelegationToken(DelegationToken appToken) {
+  public void setRMDelegationToken(Token appToken) {
     maybeInitBuilder();
     if (appToken == null) 
       builder.clearToken();
@@ -99,11 +99,11 @@ ProtoBase<GetDelegationTokenResponseProto> implements GetDelegationTokenResponse
   }
 
 
-  private DelegationTokenPBImpl convertFromProtoFormat(TokenProto p) {
-    return new DelegationTokenPBImpl(p);
+  private TokenPBImpl convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
   }
 
-  private TokenProto convertToProtoFormat(DelegationToken t) {
-    return ((DelegationTokenPBImpl)t).getProto();
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl)t).getProto();
   }
 }

+ 9 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RenewDelegationTokenRequestPBImpl.java

@@ -21,9 +21,9 @@ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenReque
 import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProtoOrBuilder;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.api.records.impl.pb.DelegationTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 
 public class RenewDelegationTokenRequestPBImpl extends
     ProtoBase<RenewDelegationTokenRequestProto> implements
@@ -43,10 +43,10 @@ public class RenewDelegationTokenRequestPBImpl extends
     this.viaProto = true;
   }
 
-  DelegationToken token;
+  Token token;
 
   @Override
-  public DelegationToken getDelegationToken() {
+  public Token getDelegationToken() {
     RenewDelegationTokenRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (this.token != null) {
       return this.token;
@@ -56,7 +56,7 @@ public class RenewDelegationTokenRequestPBImpl extends
   }
 
   @Override
-  public void setDelegationToken(DelegationToken token) {
+  public void setDelegationToken(Token token) {
     maybeInitBuilder();
     if (token == null) 
       builder.clearToken();
@@ -94,11 +94,11 @@ public class RenewDelegationTokenRequestPBImpl extends
   }
 
 
-  private DelegationTokenPBImpl convertFromProtoFormat(TokenProto p) {
-    return new DelegationTokenPBImpl(p);
+  private TokenPBImpl convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
   }
 
-  private TokenProto convertToProtoFormat(DelegationToken t) {
-    return ((DelegationTokenPBImpl)t).getProto();
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl)t).getProto();
   }
 }

+ 9 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainerRequestPBImpl.java

@@ -22,10 +22,10 @@ package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerLaunchContextPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProtoOrBuilder;
@@ -39,7 +39,7 @@ public class StartContainerRequestPBImpl extends ProtoBase<StartContainerRequest
   
   private ContainerLaunchContext containerLaunchContext = null;
 
-  private ContainerToken containerToken = null;
+  private Token containerToken = null;
   
   public StartContainerRequestPBImpl() {
     builder = StartContainerRequestProto.newBuilder();
@@ -104,7 +104,7 @@ public class StartContainerRequestPBImpl extends ProtoBase<StartContainerRequest
   }
 
   @Override
-  public ContainerToken getContainerToken() {
+  public Token getContainerToken() {
     StartContainerRequestProtoOrBuilder p = viaProto ? proto : builder;
     if (this.containerToken != null) {
       return this.containerToken;
@@ -117,7 +117,7 @@ public class StartContainerRequestPBImpl extends ProtoBase<StartContainerRequest
   }
 
   @Override
-  public void setContainerToken(ContainerToken containerToken) {
+  public void setContainerToken(Token containerToken) {
     maybeInitBuilder();
     if(containerToken == null) {
       builder.clearContainerToken();
@@ -135,11 +135,11 @@ public class StartContainerRequestPBImpl extends ProtoBase<StartContainerRequest
 
 
 
-  private ContainerTokenPBImpl convertFromProtoFormat(TokenProto containerProto) {
-    return new ContainerTokenPBImpl(containerProto);
+  private TokenPBImpl convertFromProtoFormat(TokenProto containerProto) {
+    return new TokenPBImpl(containerProto);
   }
 
-  private TokenProto convertToProtoFormat(ContainerToken container) {
-    return ((ContainerTokenPBImpl)container).getProto();
+  private TokenProto convertToProtoFormat(Token container) {
+    return ((TokenPBImpl)container).getProto();
   }
 }  

+ 14 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java

@@ -51,9 +51,10 @@ import org.apache.hadoop.yarn.util.Records;
 public abstract class ApplicationReport {
 
   @Private
+  @Stable
   public static ApplicationReport newInstance(ApplicationId applicationId,
       ApplicationAttemptId applicationAttemptId, String user, String queue,
-      String name, String host, int rpcPort, ClientToken clientToken,
+      String name, String host, int rpcPort, Token clientToken,
       YarnApplicationState state, String diagnostics, String url,
       long startTime, long finishTime, FinalApplicationStatus finalStatus,
       ApplicationResourceUsageReport appResources, String origTrackingUrl,
@@ -170,16 +171,26 @@ public abstract class ApplicationReport {
   /**
    * Get the <em>client token</em> for communicating with the
    * <code>ApplicationMaster</code>.
+   * <p>
+   * <code>ClientToken</code> is the security token used by the AMs to verify
+   * authenticity of any <code>client</code>.
+   * </p>
+   *
+   * <p>
+   * The <code>ResourceManager</code>, provides a secure token (via
+   * {@link ApplicationReport#getClientToken()}) which is verified by the
+   * ApplicationMaster when the client directly talks to an AM.
+   * </p>
    * @return <em>client token</em> for communicating with the
    * <code>ApplicationMaster</code>
    */
   @Public
   @Stable
-  public abstract ClientToken getClientToken();
+  public abstract Token getClientToken();
 
   @Private
   @Unstable
-  public abstract void setClientToken(ClientToken clientToken);
+  public abstract void setClientToken(Token clientToken);
 
   /**
    * Get the <code>YarnApplicationState</code> of the application.

+ 0 - 39
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ClientToken.java

@@ -1,39 +0,0 @@
-/**
- * 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.api.records;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Stable;
-
-/**
- * <p>
- * <code>ClientToken</code> is the security token used by the AMs to verify
- * authenticity of any <code>client</code>.
- * </p>
- * 
- * <p>
- * The <code>ResourceManager</code>, provides a secure token (via
- * {@link ApplicationReport#getClientToken()}) which is verified by the
- * ApplicationMaster when the client directly talks to an AM.
- * </p>
- * 
- */
-@Public
-@Stable
-public interface ClientToken extends Token {}

+ 18 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java

@@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.util.Records;
  *     <li>{@link Priority} at which the container was allocated.</li>
  *     <li>{@link ContainerState} of the container.</li>
  *     <li>
- *       {@link ContainerToken} of the container, used to securely verify 
+ *       Container Token {@link Token} of the container, used to securely verify
  *       authenticity of the allocation. 
  *     </li>
  *     <li>{@link ContainerStatus} of the container.</li>
@@ -69,7 +69,7 @@ public abstract class Container implements Comparable<Container> {
   @Private
   public static Container newInstance(ContainerId containerId, NodeId nodeId,
       String nodeHttpAddress, Resource resource, Priority priority,
-      ContainerToken containerToken) {
+      Token containerToken) {
     Container container = Records.newRecord(Container.class);
     container.setId(containerId);
     container.setNodeId(nodeId);
@@ -142,13 +142,27 @@ public abstract class Container implements Comparable<Container> {
   
   /**
    * Get the <code>ContainerToken</code> for the container.
+   * <p><code>ContainerToken</code> is the security token used by the framework
+   * to verify authenticity of any <code>Container</code>.</p>
+   *
+   * <p>The <code>ResourceManager</code>, on container allocation provides a
+   * secure token which is verified by the <code>NodeManager</code> on
+   * container launch.</p>
+   *
+   * <p>Applications do not need to care about <code>ContainerToken</code>, they
+   * are transparently handled by the framework - the allocated
+   * <code>Container</code> includes the <code>ContainerToken</code>.</p>
+   *
+   * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
+   * @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
+   *
    * @return <code>ContainerToken</code> for the container
    */
   @Public
   @Stable
-  public abstract ContainerToken getContainerToken();
+  public abstract Token getContainerToken();
   
   @Private
   @Unstable
-  public abstract void setContainerToken(ContainerToken containerToken);
+  public abstract void setContainerToken(Token containerToken);
 }

+ 0 - 43
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerToken.java

@@ -1,43 +0,0 @@
-/**
- * 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.api.records;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Stable;
-import org.apache.hadoop.yarn.api.AMRMProtocol;
-import org.apache.hadoop.yarn.api.ContainerManager;
-
-/**
- * <p><code>ContainerToken</code> is the security token used by the framework
- * to verify authenticity of any <code>Container</code>.</p>
- *
- * <p>The <code>ResourceManager</code>, on container allocation provides a
- * secure token which is verified by the <code>NodeManager</code> on 
- * container launch.</p>
- * 
- * <p>Applications do not need to care about <code>ContainerToken</code>, they
- * are transparently handled by the framework - the allocated 
- * <code>Container</code> includes the <code>ContainerToken</code>.</p>
- * 
- * @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
- * @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
- */
-@Public
-@Stable
-public interface ContainerToken extends Token {}

+ 0 - 32
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/DelegationToken.java

@@ -1,32 +0,0 @@
-/**
- * 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.api.records;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
-
-/**
- * The Delegation tokens have a identifier which maps to
- * {@link AbstractDelegationTokenIdentifier}.
- * 
- */
-@Public
-@Evolving
-public interface DelegationToken extends Token {}

+ 22 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Token.java

@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.yarn.util.Records;
 
 /**
  * <p><code>Token</code> is the security entity used by the framework
@@ -30,18 +31,30 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
  */
 @Public
 @Stable
-public interface Token {
+public abstract class Token {
+
+  @Private
+  public static Token newInstance(byte[] identifier, String kind, byte[] password,
+      String service) {
+    Token token = Records.newRecord(Token.class);
+    token.setIdentifier(ByteBuffer.wrap(identifier));
+    token.setKind(kind);
+    token.setPassword(ByteBuffer.wrap(password));
+    token.setService(service);
+    return token;
+  }
+
   /**
    * Get the token identifier.
    * @return token identifier
    */
   @Public
   @Stable
-  ByteBuffer getIdentifier();
+  public abstract ByteBuffer getIdentifier();
   
   @Private
   @Stable
-  void setIdentifier(ByteBuffer identifier);
+  public abstract void setIdentifier(ByteBuffer identifier);
 
   /**
    * Get the token password
@@ -49,11 +62,11 @@ public interface Token {
    */
   @Public
   @Stable
-  ByteBuffer getPassword();
+  public abstract ByteBuffer getPassword();
   
   @Private
   @Stable
-  void setPassword(ByteBuffer password);
+  public abstract void setPassword(ByteBuffer password);
 
   /**
    * Get the token kind.
@@ -61,11 +74,11 @@ public interface Token {
    */
   @Public
   @Stable
-  String getKind();
+  public abstract String getKind();
   
   @Private
   @Stable
-  void setKind(String kind);
+  public abstract void setKind(String kind);
 
   /**
    * Get the service to which the token is allocated.
@@ -73,10 +86,10 @@ public interface Token {
    */
   @Public
   @Stable
-  String getService();
+  public abstract String getService();
 
   @Private
   @Stable
-  void setService(String service);
+  public abstract void setService(String service);
 
 }

+ 9 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java

@@ -23,8 +23,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
-import org.apache.hadoop.yarn.api.records.ClientToken;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
@@ -42,7 +42,7 @@ public class ApplicationReportPBImpl extends ApplicationReport {
 
   private ApplicationId applicationId;
   private ApplicationAttemptId currentApplicationAttemptId;
-  private ClientToken clientToken = null;
+  private Token clientToken = null;
 
   public ApplicationReportPBImpl() {
     builder = ApplicationReportProto.newBuilder();
@@ -160,7 +160,7 @@ public class ApplicationReportPBImpl extends ApplicationReport {
   }
 
   @Override
-  public ClientToken getClientToken() {
+  public Token getClientToken() {
     ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
     if (this.clientToken != null) {
       return this.clientToken;
@@ -309,7 +309,7 @@ public class ApplicationReportPBImpl extends ApplicationReport {
   }
 
   @Override
-  public void setClientToken(ClientToken clientToken) {
+  public void setClientToken(Token clientToken) {
     maybeInitBuilder();
     if (clientToken == null) 
       builder.clearClientToken();
@@ -413,7 +413,7 @@ public class ApplicationReportPBImpl extends ApplicationReport {
       builder.setCurrentApplicationAttemptId(convertToProtoFormat(this.currentApplicationAttemptId));
     }
     if (this.clientToken != null
-        && !((ClientTokenPBImpl) this.clientToken).getProto().equals(
+        && !((TokenPBImpl) this.clientToken).getProto().equals(
             builder.getClientToken())) {
       builder.setClientToken(convertToProtoFormat(this.clientToken));
     }
@@ -476,11 +476,11 @@ public class ApplicationReportPBImpl extends ApplicationReport {
     return ProtoUtils.convertToProtoFormat(s);
   }
 
-  private ClientTokenPBImpl convertFromProtoFormat(TokenProto p) {
-    return new ClientTokenPBImpl(p);
+  private TokenPBImpl convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
   }
 
-  private TokenProto convertToProtoFormat(ClientToken t) {
-    return ((ClientTokenPBImpl)t).getProto();
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl)t).getProto();
   }
 }

+ 0 - 33
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ClientTokenPBImpl.java

@@ -1,33 +0,0 @@
-/**
- * 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.api.records.impl.pb;
-
-import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
-import org.apache.hadoop.yarn.api.records.ClientToken;
-
-public class ClientTokenPBImpl extends TokenPBImpl implements ClientToken {
-
-  public ClientTokenPBImpl() {
-    super();
-  }
-
-  public ClientTokenPBImpl(TokenProto p) {
-    super(p);
-  }
-}

+ 9 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java

@@ -21,10 +21,10 @@ package org.apache.hadoop.yarn.api.records.impl.pb;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProtoOrBuilder;
@@ -42,7 +42,7 @@ public class ContainerPBImpl extends Container {
   private NodeId nodeId = null;
   private Resource resource = null;
   private Priority priority = null;
-  private ContainerToken containerToken = null;
+  private Token containerToken = null;
   
   public ContainerPBImpl() {
     builder = ContainerProto.newBuilder();
@@ -98,7 +98,7 @@ public class ContainerPBImpl extends Container {
       builder.setPriority(convertToProtoFormat(this.priority));
     }
     if (this.containerToken != null
-        && !((ContainerTokenPBImpl) this.containerToken).getProto().equals(
+        && !((TokenPBImpl) this.containerToken).getProto().equals(
             builder.getContainerToken())) {
       builder.setContainerToken(convertToProtoFormat(this.containerToken));
     }
@@ -224,7 +224,7 @@ public class ContainerPBImpl extends Container {
   }
 
   @Override
-  public ContainerToken getContainerToken() {
+  public Token getContainerToken() {
     ContainerProtoOrBuilder p = viaProto ? proto : builder;
     if (this.containerToken != null) {
       return this.containerToken;
@@ -237,7 +237,7 @@ public class ContainerPBImpl extends Container {
   }
 
   @Override
-  public void setContainerToken(ContainerToken containerToken) {
+  public void setContainerToken(Token containerToken) {
     maybeInitBuilder();
     if (containerToken == null) 
       builder.clearContainerToken();
@@ -276,12 +276,12 @@ public class ContainerPBImpl extends Container {
     return ((PriorityPBImpl)p).getProto();
   }
   
-  private ContainerTokenPBImpl convertFromProtoFormat(TokenProto p) {
-    return new ContainerTokenPBImpl(p);
+  private TokenPBImpl convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
   }
 
-  private TokenProto convertToProtoFormat(ContainerToken t) {
-    return ((ContainerTokenPBImpl)t).getProto();
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl)t).getProto();
   }
 
   public String toString() {

+ 0 - 33
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerTokenPBImpl.java

@@ -1,33 +0,0 @@
-/**
- * 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.api.records.impl.pb;
-
-import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
-
-public class ContainerTokenPBImpl extends TokenPBImpl implements ContainerToken {
-
-  public ContainerTokenPBImpl() {
-    super();
-  }
-
-  public ContainerTokenPBImpl(TokenProto p) {
-    super(p);
-  }
-}

+ 0 - 34
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/DelegationTokenPBImpl.java

@@ -1,34 +0,0 @@
-/**
- * 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.api.records.impl.pb;
-
-import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
-
-public class DelegationTokenPBImpl extends TokenPBImpl implements
-    DelegationToken {
-
-  public DelegationTokenPBImpl() {
-    super();
-  }
-
-  public DelegationTokenPBImpl(TokenProto p) {
-    super(p);
-  }
-}

+ 27 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/TokenPBImpl.java

@@ -22,11 +22,12 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProtoOrBuilder;
-import org.apache.hadoop.yarn.api.records.ProtoBase;
 import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.util.ProtoUtils;
 
-public class TokenPBImpl extends ProtoBase<TokenProto> implements
-    Token {
+import com.google.protobuf.ByteString;
+
+public class TokenPBImpl extends Token {
   private TokenProto proto = TokenProto.getDefaultInstance();
   private TokenProto.Builder builder = null;
   private boolean viaProto = false;
@@ -50,6 +51,29 @@ public class TokenPBImpl extends ProtoBase<TokenProto> implements
     return proto;
   }
 
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  protected final ByteBuffer convertFromProtoFormat(ByteString byteString) {
+    return ProtoUtils.convertFromProtoFormat(byteString);
+  }
+
+  protected final ByteString convertToProtoFormat(ByteBuffer byteBuffer) {
+    return ProtoUtils.convertToProtoFormat(byteBuffer);
+  }
+
   private synchronized void mergeLocalToBuilder() {
     if (this.identifier != null) {
       builder.setIdentifier(convertToProtoFormat(this.identifier));

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClient.java

@@ -28,8 +28,8 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.service.Service;
 
@@ -69,7 +69,7 @@ public interface NMClient extends Service {
    * @throws IOException
    */
   void stopContainer(ContainerId containerId, NodeId nodeId,
-      ContainerToken containerToken) throws YarnRemoteException, IOException;
+      Token containerToken) throws YarnRemoteException, IOException;
 
   /**
    * <p>Query the status of a container.</p>
@@ -83,7 +83,7 @@ public interface NMClient extends Service {
    * @throws IOException
    */
   ContainerStatus getContainerStatus(ContainerId containerId, NodeId nodeId,
-      ContainerToken containerToken) throws YarnRemoteException, IOException;
+      Token containerToken) throws YarnRemoteException, IOException;
 
   /**
    * <p>Set whether the containers that are started by this client, and are

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java

@@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AbstractEvent;
@@ -288,7 +288,7 @@ public class NMClientAsync extends AbstractService {
   }
 
   public void stopContainer(ContainerId containerId, NodeId nodeId,
-      ContainerToken containerToken) {
+      Token containerToken) {
     if (containers.get(containerId) == null) {
       callbackHandler.onStopContainerError(containerId,
           RPCUtil.getRemoteException("Container " + containerId +
@@ -305,7 +305,7 @@ public class NMClientAsync extends AbstractService {
   }
 
   public void getContainerStatus(ContainerId containerId, NodeId nodeId,
-      ContainerToken containerToken) {
+      Token containerToken) {
     try {
       events.put(new ContainerEvent(containerId, nodeId, containerToken,
           ContainerEventType.QUERY_CONTAINER));
@@ -343,10 +343,10 @@ public class NMClientAsync extends AbstractService {
       extends AbstractEvent<ContainerEventType>{
     private ContainerId containerId;
     private NodeId nodeId;
-    private ContainerToken containerToken;
+    private Token containerToken;
 
     public ContainerEvent(ContainerId containerId, NodeId nodeId,
-        ContainerToken containerToken, ContainerEventType type) {
+        Token containerToken, ContainerEventType type) {
       super(type);
       this.containerId = containerId;
       this.nodeId = nodeId;
@@ -361,7 +361,7 @@ public class NMClientAsync extends AbstractService {
       return nodeId;
     }
 
-    public ContainerToken getContainerToken() {
+    public Token getContainerToken() {
       return containerToken;
     }
   }

+ 9 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java

@@ -32,7 +32,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.ContainerManager;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
@@ -43,7 +42,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
@@ -122,11 +121,11 @@ public class NMClientImpl extends AbstractService implements NMClient {
   protected static class StartedContainer {
     private ContainerId containerId;
     private NodeId nodeId;
-    private ContainerToken containerToken;
+    private Token containerToken;
     private boolean stopped;
 
     public StartedContainer(ContainerId containerId, NodeId nodeId,
-        ContainerToken containerToken) {
+        Token containerToken) {
       this.containerId = containerId;
       this.nodeId = nodeId;
       this.containerToken = containerToken;
@@ -141,7 +140,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
       return nodeId;
     }
 
-    public ContainerToken getContainerToken() {
+    public Token getContainerToken() {
       return containerToken;
     }
   }
@@ -149,11 +148,11 @@ public class NMClientImpl extends AbstractService implements NMClient {
   protected static final class NMCommunicator extends AbstractService {
     private ContainerId containerId;
     private NodeId nodeId;
-    private ContainerToken containerToken;
+    private Token containerToken;
     private ContainerManager containerManager;
 
     public NMCommunicator(ContainerId containerId, NodeId nodeId,
-        ContainerToken containerToken) {
+        Token containerToken) {
       super(NMCommunicator.class.getName());
       this.containerId = containerId;
       this.nodeId = nodeId;
@@ -171,7 +170,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
       UserGroupInformation currentUser =
           UserGroupInformation.createRemoteUser(containerId.toString());
 
-      Token<ContainerTokenIdentifier> token =
+      org.apache.hadoop.security.token.Token<ContainerTokenIdentifier> token =
           ProtoUtils.convertFromProtoFormat(containerToken, containerAddress);
       currentUser.addToken(token);
 
@@ -316,7 +315,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
 
   @Override
   public void stopContainer(ContainerId containerId, NodeId nodeId,
-      ContainerToken containerToken) throws YarnRemoteException, IOException {
+      Token containerToken) throws YarnRemoteException, IOException {
     StartedContainer startedContainer = getStartedContainer(containerId);
     if (startedContainer == null) {
       throw RPCUtil.getRemoteException("Container " + containerId +
@@ -348,7 +347,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
 
   @Override
   public ContainerStatus getContainerStatus(ContainerId containerId,
-      NodeId nodeId, ContainerToken containerToken)
+      NodeId nodeId, Token containerToken)
           throws YarnRemoteException, IOException {
     NMCommunicator nmCommunicator = null;
     try {

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClient.java

@@ -28,10 +28,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.service.Service;
@@ -179,12 +179,12 @@ public interface YarnClient extends Service {
    * @param renewer
    *          Address of the renewer who can renew these tokens when needed by
    *          securely talking to YARN.
-   * @return a delegation token ({@link DelegationToken}) that can be used to
+   * @return a delegation token ({@link Token}) that can be used to
    *         talk to YARN
    * @throws YarnRemoteException
    * @throws IOException
    */
-  DelegationToken getRMDelegationToken(Text renewer)
+  Token getRMDelegationToken(Text renewer)
       throws YarnRemoteException, IOException;
 
   /**

+ 2 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java

@@ -50,15 +50,14 @@ import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.util.Records;
@@ -215,7 +214,7 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
   }
 
   @Override
-  public DelegationToken getRMDelegationToken(Text renewer)
+  public Token getRMDelegationToken(Text renewer)
       throws YarnRemoteException, IOException {
     /* get the token from RM */
     GetDelegationTokenRequest rmDTRequest =

+ 9 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java

@@ -45,7 +45,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
@@ -63,7 +63,7 @@ public class TestNMClientAsync {
 
   private NMClientAsync asyncClient;
   private NodeId nodeId;
-  private ContainerToken containerToken;
+  private Token containerToken;
 
   @Test (timeout = 30000)
   public void testNMClientAsync() throws Exception {
@@ -368,10 +368,10 @@ public class TestNMClientAsync {
             any(ContainerLaunchContext.class))).thenReturn(
                 Collections.<String, ByteBuffer>emptyMap());
         when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class),
-            any(ContainerToken.class))).thenReturn(
+            any(Token.class))).thenReturn(
                 recordFactory.newRecordInstance(ContainerStatus.class));
         doNothing().when(client).stopContainer(any(ContainerId.class),
-            any(NodeId.class), any(ContainerToken.class));
+            any(NodeId.class), any(Token.class));
         break;
       case 1:
         doThrow(RPCUtil.getRemoteException("Start Exception")).when(client)
@@ -379,21 +379,21 @@ public class TestNMClientAsync {
                 any(ContainerLaunchContext.class));
         doThrow(RPCUtil.getRemoteException("Query Exception")).when(client)
             .getContainerStatus(any(ContainerId.class), any(NodeId.class),
-                any(ContainerToken.class));
+                any(Token.class));
         doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client)
             .stopContainer(any(ContainerId.class), any(NodeId.class),
-                any(ContainerToken.class));
+                any(Token.class));
         break;
       case 2:
         when(client.startContainer(any(Container.class),
             any(ContainerLaunchContext.class))).thenReturn(
                 Collections.<String, ByteBuffer>emptyMap());
         when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class),
-            any(ContainerToken.class))).thenReturn(
+            any(Token.class))).thenReturn(
                 recordFactory.newRecordInstance(ContainerStatus.class));
         doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client)
             .stopContainer(any(ContainerId.class), any(NodeId.class),
-                any(ContainerToken.class));
+                any(Token.class));
     }
     return client;
   }
@@ -532,9 +532,8 @@ public class TestNMClientAsync {
     ContainerId containerId = ContainerId.newInstance(attemptId, i);
     nodeId = NodeId.newInstance("localhost", 0);
     // Create an empty record
-    containerToken = recordFactory.newRecordInstance(ContainerToken.class);
+    containerToken = recordFactory.newRecordInstance(Token.class);
     return BuilderUtils.newContainer(containerId, nodeId, null, null, null,
       containerToken);
   }
-
 }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/RMDelegationTokenIdentifier.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecret
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.util.BuilderUtils;
@@ -164,7 +163,8 @@ public class RMDelegationTokenIdentifier extends AbstractDelegationTokenIdentifi
           .getRenewer().toString();
     }
     
-    private static DelegationToken convertToProtoToken(Token<?> token) {
+    private static org.apache.hadoop.yarn.api.records.Token
+        convertToProtoToken(Token<?> token) {
       return BuilderUtils.newDelegationToken(
           token.getIdentifier(), token.getKind().toString(),
           token.getPassword(), token.getService().toString());

+ 11 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java

@@ -38,14 +38,11 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ClientToken;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
@@ -169,7 +166,7 @@ public class BuilderUtils {
     return cId;
   }
 
-  public static ContainerToken newContainerToken(ContainerId cId, String host,
+  public static Token newContainerToken(ContainerId cId, String host,
       int port, String user, Resource r, long expiryTime, int masterKeyId,
       byte[] password, long rmIdentifier) throws IOException {
     ContainerTokenIdentifier identifier =
@@ -217,7 +214,7 @@ public class BuilderUtils {
 
   public static Container newContainer(ContainerId containerId, NodeId nodeId,
       String nodeHttpAddress, Resource resource, Priority priority,
-      ContainerToken containerToken) {
+      Token containerToken) {
     Container container = recordFactory.newRecordInstance(Container.class);
     container.setId(containerId);
     container.setNodeId(nodeId);
@@ -238,31 +235,31 @@ public class BuilderUtils {
     return token;
   }
 
-  public static DelegationToken newDelegationToken(byte[] identifier,
+  public static Token newDelegationToken(byte[] identifier,
       String kind, byte[] password, String service) {
-    return newToken(DelegationToken.class, identifier, kind, password, service);
+    return newToken(Token.class, identifier, kind, password, service);
   }
 
-  public static ClientToken newClientToken(byte[] identifier, String kind,
+  public static Token newClientToken(byte[] identifier, String kind,
       byte[] password, String service) {
-    return newToken(ClientToken.class, identifier, kind, password, service);
+    return newToken(Token.class, identifier, kind, password, service);
   }
 
-  public static ContainerToken newContainerToken(NodeId nodeId,
+  public static Token newContainerToken(NodeId nodeId,
       byte[] password, ContainerTokenIdentifier tokenIdentifier) {
     // RPC layer client expects ip:port as service for tokens
     InetSocketAddress addr =
         NetUtils.createSocketAddrForHost(nodeId.getHost(), nodeId.getPort());
     // NOTE: use SecurityUtil.setTokenService if this becomes a "real" token
-    ContainerToken containerToken =
-        newToken(ContainerToken.class, tokenIdentifier.getBytes(),
+    Token containerToken =
+        newToken(Token.class, tokenIdentifier.getBytes(),
           ContainerTokenIdentifier.KIND.toString(), password, SecurityUtil
             .buildTokenService(addr).toString());
     return containerToken;
   }
 
   public static ContainerTokenIdentifier newContainerTokenIdentifier(
-      ContainerToken containerToken) throws IOException {
+      Token containerToken) throws IOException {
     org.apache.hadoop.security.token.Token<ContainerTokenIdentifier> token =
         new org.apache.hadoop.security.token.Token<ContainerTokenIdentifier>(
             containerToken.getIdentifier()
@@ -318,7 +315,7 @@ public class BuilderUtils {
   public static ApplicationReport newApplicationReport(
       ApplicationId applicationId, ApplicationAttemptId applicationAttemptId,
       String user, String queue, String name, String host, int rpcPort,
-      ClientToken clientToken, YarnApplicationState state, String diagnostics,
+      Token clientToken, YarnApplicationState state, String diagnostics,
       String url, long startTime, long finishTime,
       FinalApplicationStatus finalStatus,
       ApplicationResourceUsageReport appResources, String origTrackingUrl,

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

@@ -42,9 +42,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -101,7 +101,7 @@ public class TestContainerLaunchRPC {
       ContainerTokenIdentifier containerTokenIdentifier =
           new ContainerTokenIdentifier(containerId, "localhost", "user",
             resource, System.currentTimeMillis() + 10000, 42, 42);
-      ContainerToken containerToken =
+      Token containerToken =
           BuilderUtils.newContainerToken(nodeId, "password".getBytes(),
             containerTokenIdentifier);
 

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java

@@ -44,9 +44,9 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -129,7 +129,7 @@ public class TestRPC {
     ContainerTokenIdentifier containerTokenIdentifier =
         new ContainerTokenIdentifier(containerId, "localhost", "user",
           resource, System.currentTimeMillis() + 10000, 42, 42);
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(nodeId, "password".getBytes(),
           containerTokenIdentifier);
     scRequest.setContainerToken(containerToken);
@@ -179,7 +179,7 @@ public class TestRPC {
     @Override
     public StartContainerResponse startContainer(StartContainerRequest request) 
         throws YarnRemoteException {
-      ContainerToken containerToken = request.getContainerToken();
+      Token containerToken = request.getContainerToken();
       ContainerTokenIdentifier tokenId = null;
 
       try {

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -52,7 +52,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -401,7 +400,7 @@ public class ContainerManagerImpl extends CompositeService implements
     }
 
     ContainerLaunchContext launchContext = request.getContainerLaunchContext();
-    ContainerToken token = request.getContainerToken();
+    org.apache.hadoop.yarn.api.records.Token token = request.getContainerToken();
 
     ContainerTokenIdentifier tokenIdentifier = null;
     try {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java

@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -135,7 +135,7 @@ public class TestEventFlow {
     String user = "testing";
     String host = "127.0.0.1";
     int port = 1234;
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cID, host, port, user, r,
           System.currentTimeMillis() + 10000L, 123, "password".getBytes(),
           SIMULATED_RM_IDENTIFIER);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java

@@ -46,12 +46,12 @@ 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.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -134,7 +134,7 @@ public class TestNodeManagerReboot {
     Resource resource = Records.newRecord(Resource.class);
     resource.setMemory(1024);
     NodeId nodeId = BuilderUtils.newNodeId("127.0.0.1", 12345);
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cId, nodeId.getHost(), nodeId.getPort(),
           user, resource, System.currentTimeMillis() + 10000L, 123,
           "password".getBytes(), 0);

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java

@@ -48,12 +48,12 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -177,7 +177,7 @@ public class TestNodeManagerShutdown {
     List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
     containerLaunchContext.setCommands(commands);
     Resource resource = BuilderUtils.newResource(1024, 1);
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cId, nodeId.getHost(), nodeId.getPort(),
           user, resource, System.currentTimeMillis() + 10000L, 123,
           "password".getBytes(), 0);
@@ -257,7 +257,7 @@ public class TestNodeManagerShutdown {
     fileWriter.close();
     return scriptFile;
   }
-  
+
   class TestNodeManager extends NodeManager {
 
     @Override

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java

@@ -44,11 +44,11 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
@@ -136,7 +136,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     containerLaunchContext.setLocalResources(localResources);
     Resource r = BuilderUtils.newResource(512, 1);
     int port = 12345;
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
           port, user, r, System.currentTimeMillis() + 10000L, 123,
           "password".getBytes(), super.DUMMY_RM_IDENTIFIER);
@@ -229,7 +229,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     containerLaunchContext.setCommands(commands);
     Resource r = BuilderUtils.newResource(100, 1);
     int port = 12345;
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
           port, user, r, System.currentTimeMillis() + 10000L, 123,
           "password".getBytes(), super.DUMMY_RM_IDENTIFIER);
@@ -337,7 +337,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
 	  containerLaunchContext.setCommands(commands);
     Resource r = BuilderUtils.newResource(100, 1);
     int port = 12345;
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
           port, user, r, System.currentTimeMillis() + 10000L, 123,
           "password".getBytes(), super.DUMMY_RM_IDENTIFIER);
@@ -426,7 +426,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Resource r = BuilderUtils.newResource(100, 1);
     int port = 12345;
 
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
           port, user, r, System.currentTimeMillis() + 10000L, 123,
           "password".getBytes(), super.DUMMY_RM_IDENTIFIER);
@@ -516,7 +516,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
         recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest1.setContainerLaunchContext(containerLaunchContext);
     
-    ContainerToken containerToken1 =
+    Token containerToken1 =
         BuilderUtils.newContainerToken(cId1, host, port, user, mockResource,
           System.currentTimeMillis() + 10000, 123, "password".getBytes(), 
           (long) ResourceManagerConstants.RM_INVALID_IDENTIFIER);
@@ -541,7 +541,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     StartContainerRequest startRequest2 =
         recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest2.setContainerLaunchContext(containerLaunchContext);
-    ContainerToken containerToken2 =
+    Token containerToken2 =
         BuilderUtils.newContainerToken(cId1, host, port, user, mockResource,
           System.currentTimeMillis() + 10000, 123, "password".getBytes(),
           super.DUMMY_RM_IDENTIFIER);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java

@@ -45,11 +45,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -578,7 +578,7 @@ public class TestContainer {
       ContainerTokenIdentifier identifier =
           new ContainerTokenIdentifier(cId, "127.0.0.1", user, resource,
             currentTime + 10000L, 123, currentTime);
-      ContainerToken token =
+      Token token =
           BuilderUtils.newContainerToken(BuilderUtils.newNodeId(host, port),
             "password".getBytes(), identifier);
       when(mockContainer.getContainerToken()).thenReturn(token);

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

@@ -49,11 +49,11 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
@@ -233,7 +233,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     Resource r = BuilderUtils.newResource(1024, 1);
     StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
           port, user, r, System.currentTimeMillis() + 10000L, 1234,
           "password".getBytes(), super.DUMMY_RM_IDENTIFIER);
@@ -369,7 +369,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
     containerLaunchContext.setCommands(commands);
     Resource r = BuilderUtils.newResource(1024, 1);
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
           port, user, r, System.currentTimeMillis() + 10000L, 123,
           "password".getBytes(), super.DUMMY_RM_IDENTIFIER);

+ 13 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java

@@ -18,10 +18,19 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation;
 
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.*;
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyMap;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
@@ -58,12 +67,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -742,7 +751,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     commands.add(scriptFile.getAbsolutePath());
     containerLaunchContext.setCommands(commands);
     Resource r = BuilderUtils.newResource(100 * 1024 * 1024, 1);
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cId, "127.0.0.1", 1234, user, r,
           System.currentTimeMillis() + 10000L, 123, "password".getBytes(),
           super.DUMMY_RM_IDENTIFIER);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java

@@ -48,11 +48,11 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -229,7 +229,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
     StartContainerRequest startRequest =
         recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
-    ContainerToken containerToken =
+    Token containerToken =
         BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
           port, user, r, System.currentTimeMillis() + 10000L, 123,
           "password".getBytes(), super.DUMMY_RM_IDENTIFIER);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java

@@ -32,7 +32,7 @@ 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.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -179,7 +179,7 @@ public class TestNMWebServer {
       ContainerLaunchContext launchContext =
           recordFactory.newRecordInstance(ContainerLaunchContext.class);
       long currentTime = System.currentTimeMillis();
-      ContainerToken containerToken =
+      Token containerToken =
           BuilderUtils.newContainerToken(containerId, "127.0.0.1", 1234, user,
             BuilderUtils.newResource(1024, 1), currentTime + 10000L, 123,
             "password".getBytes(), currentTime);

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

@@ -67,7 +67,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -538,7 +537,7 @@ public class ClientRMService extends AbstractService implements
             "Delegation Token can be renewed only with kerberos authentication");
       }
       
-      DelegationToken protoToken = request.getDelegationToken();
+      org.apache.hadoop.yarn.api.records.Token protoToken = request.getDelegationToken();
       Token<RMDelegationTokenIdentifier> token = new Token<RMDelegationTokenIdentifier>(
           protoToken.getIdentifier().array(), protoToken.getPassword().array(),
           new Text(protoToken.getKind()), new Text(protoToken.getService()));
@@ -562,7 +561,7 @@ public class ClientRMService extends AbstractService implements
         throw new IOException(
             "Delegation Token can be cancelled only with kerberos authentication");
       }
-      DelegationToken protoToken = request.getDelegationToken();
+      org.apache.hadoop.yarn.api.records.Token protoToken = request.getDelegationToken();
       Token<RMDelegationTokenIdentifier> token = new Token<RMDelegationTokenIdentifier>(
           protoToken.getIdentifier().array(), protoToken.getPassword().array(),
           new Text(protoToken.getKind()), new Text(protoToken.getService()));

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

@@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ClientToken;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -431,7 +430,7 @@ public class RMAppImpl implements RMApp, Recoverable {
 
     try {
       ApplicationAttemptId currentApplicationAttemptId = null;
-      ClientToken clientToken = null;
+      org.apache.hadoop.yarn.api.records.Token clientToken = null;
       String trackingUrl = UNAVAILABLE;
       String host = UNAVAILABLE;
       String origTrackingUrl = UNAVAILABLE;

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -51,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.QueueState;
 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.api.records.Token;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
@@ -1251,7 +1251,7 @@ public class LeafQueue implements CSQueue {
   /**
    * Create <code>ContainerToken</code>, only in secure-mode
    */
-  ContainerToken createContainerToken(
+  Token createContainerToken(
       FiCaSchedulerApp application, Container container) {
     return containerTokenSecretManager.createContainerToken(
         container.getId(), container.getNodeId(),
@@ -1295,7 +1295,7 @@ public class LeafQueue implements CSQueue {
         unreserve(application, priority, node, rmContainer);
       }
 
-      ContainerToken containerToken =
+      Token containerToken =
           createContainerToken(application, container);
       if (containerToken == null) {
         // Something went wrong...

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java

@@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -157,7 +156,7 @@ public class AppSchedulable extends Schedulable {
     NodeId nodeId = node.getRMNode().getNodeID();
     ContainerId containerId = BuilderUtils.newContainerId(application
         .getApplicationAttemptId(), application.getNewContainerId());
-    ContainerToken containerToken =
+    org.apache.hadoop.yarn.api.records.Token containerToken =
         containerTokenSecretManager.createContainerToken(containerId, nodeId,
           application.getUser(), capability);
     if (containerToken == null) {

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

@@ -41,7 +41,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -549,7 +549,7 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
         NodeId nodeId = node.getRMNode().getNodeID();
         ContainerId containerId = BuilderUtils.newContainerId(application
             .getApplicationAttemptId(), application.getNewContainerId());
-        ContainerToken containerToken = null;
+        Token containerToken = null;
 
         containerToken =
             this.rmContext.getContainerTokenSecretManager()

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java

@@ -26,9 +26,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
@@ -170,7 +170,7 @@ public class RMContainerTokenSecretManager extends
    * @param capability
    * @return the container-token
    */
-  public ContainerToken
+  public Token
       createContainerToken(ContainerId containerId, NodeId nodeId,
           String appSubmitter, Resource capability) {
     byte[] password;

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java

@@ -42,10 +42,10 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -162,7 +162,7 @@ public class NodeManager implements ContainerManager {
       StartContainerRequest request) 
   throws YarnRemoteException {
 
-    ContainerToken containerToken = request.getContainerToken();
+    Token containerToken = request.getContainerToken();
     ContainerTokenIdentifier tokenId = null;
 
     try {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
@@ -74,7 +74,7 @@ public class TestApplicationMasterLauncher {
       Map<String, String> env =
           request.getContainerLaunchContext().getEnvironment();
 
-      ContainerToken containerToken = request.getContainerToken();
+      Token containerToken = request.getContainerToken();
       ContainerTokenIdentifier tokenId = null;
 
       try {

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -52,7 +52,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -244,7 +243,7 @@ public class TestClientRMService {
             new Text(owner.getUserName()), new Text(renewer.getUserName()), null);
     Token<?> token =
         new Token<RMDelegationTokenIdentifier>(tokenIdentifier, dtsm);
-    DelegationToken dToken = BuilderUtils.newDelegationToken(
+    org.apache.hadoop.yarn.api.records.Token dToken = BuilderUtils.newDelegationToken(
         token.getIdentifier(), token.getKind().toString(),
         token.getPassword(), token.getService().toString());
     RenewDelegationTokenRequest request =

+ 12 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java

@@ -57,7 +57,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -120,7 +119,7 @@ public class TestClientRMTokens {
       loggedInUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
 
       
-      DelegationToken token = getDelegationToken(loggedInUser, clientRMService,
+      org.apache.hadoop.yarn.api.records.Token token = getDelegationToken(loggedInUser, clientRMService,
           loggedInUser.getShortUserName());
       long tokenFetchTime = System.currentTimeMillis();
       LOG.info("Got delegation token at: " + tokenFetchTime);
@@ -350,14 +349,15 @@ public class TestClientRMTokens {
   
   // Get the delegation token directly as it is a little difficult to setup
   // the kerberos based rpc.
-  private DelegationToken getDelegationToken(
+  private org.apache.hadoop.yarn.api.records.Token getDelegationToken(
       final UserGroupInformation loggedInUser,
       final ClientRMProtocol clientRMService, final String renewerString)
       throws IOException, InterruptedException {
-    DelegationToken token = loggedInUser
-        .doAs(new PrivilegedExceptionAction<DelegationToken>() {
+    org.apache.hadoop.yarn.api.records.Token token = loggedInUser
+        .doAs(new PrivilegedExceptionAction<org.apache.hadoop.yarn.api.records.Token>() {
           @Override
-          public DelegationToken run() throws YarnRemoteException, IOException {
+            public org.apache.hadoop.yarn.api.records.Token run()
+                throws YarnRemoteException, IOException {
             GetDelegationTokenRequest request = Records
                 .newRecord(GetDelegationTokenRequest.class);
             request.setRenewer(renewerString);
@@ -369,7 +369,8 @@ public class TestClientRMTokens {
   }
   
   private long renewDelegationToken(final UserGroupInformation loggedInUser,
-      final ClientRMProtocol clientRMService, final DelegationToken dToken)
+      final ClientRMProtocol clientRMService,
+      final org.apache.hadoop.yarn.api.records.Token dToken)
       throws IOException, InterruptedException {
     long nextExpTime = loggedInUser.doAs(new PrivilegedExceptionAction<Long>() {
       @Override
@@ -385,7 +386,8 @@ public class TestClientRMTokens {
   }
   
   private void cancelDelegationToken(final UserGroupInformation loggedInUser,
-      final ClientRMProtocol clientRMService, final DelegationToken dToken)
+      final ClientRMProtocol clientRMService,
+      final org.apache.hadoop.yarn.api.records.Token dToken)
       throws IOException, InterruptedException {
     loggedInUser.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
@@ -399,7 +401,8 @@ public class TestClientRMTokens {
     });
   }
   
-  private ClientRMProtocol getClientRMProtocolWithDT(DelegationToken token,
+  private ClientRMProtocol getClientRMProtocolWithDT(
+      org.apache.hadoop.yarn.api.records.Token token,
       final InetSocketAddress rmAddress, String user, final Configuration conf) {
     // Maybe consider converting to Hadoop token, serialize de-serialize etc
     // before trying to renew the token.

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java

@@ -48,7 +48,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
@@ -597,7 +596,8 @@ public class TestRMRestart {
     when(request1.getRenewer()).thenReturn("renewer1");
     GetDelegationTokenResponse response1 =
         rm1.getClientRMService().getDelegationToken(request1);
-    DelegationToken delegationToken1 = response1.getRMDelegationToken();
+    org.apache.hadoop.yarn.api.records.Token delegationToken1 =
+        response1.getRMDelegationToken();
     Token<RMDelegationTokenIdentifier> token1 =
         ProtoUtils.convertFromProtoFormat(delegationToken1, null);
     RMDelegationTokenIdentifier dtId1 = token1.decodeIdentifier();
@@ -635,7 +635,8 @@ public class TestRMRestart {
     when(request2.getRenewer()).thenReturn("renewer2");
     GetDelegationTokenResponse response2 =
         rm1.getClientRMService().getDelegationToken(request2);
-    DelegationToken delegationToken2 = response2.getRMDelegationToken();
+    org.apache.hadoop.yarn.api.records.Token delegationToken2 =
+        response2.getRMDelegationToken();
     Token<RMDelegationTokenIdentifier> token2 =
         ProtoUtils.convertFromProtoFormat(delegationToken2, null);
     RMDelegationTokenIdentifier dtId2 = token2.decodeIdentifier();

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java

@@ -36,7 +36,6 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueInfo;

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java

@@ -57,7 +57,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ClientToken;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
@@ -222,7 +221,7 @@ public class TestClientTokens {
     GetApplicationReportResponse reportResponse =
         rm.getClientRMService().getApplicationReport(request);
     ApplicationReport appReport = reportResponse.getApplicationReport();
-    ClientToken clientToken = appReport.getClientToken();
+    org.apache.hadoop.yarn.api.records.Token clientToken = appReport.getClientToken();
 
     // Wait till AM is 'launched'
     int waitTime = 0;

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -99,7 +98,8 @@ public class TestRMDelegationTokens {
     when(request.getRenewer()).thenReturn("renewer1");
     GetDelegationTokenResponse response =
         rm1.getClientRMService().getDelegationToken(request);
-    DelegationToken delegationToken = response.getRMDelegationToken();
+    org.apache.hadoop.yarn.api.records.Token delegationToken =
+        response.getRMDelegationToken();
     Token<RMDelegationTokenIdentifier> token1 =
         ProtoUtils.convertFromProtoFormat(delegationToken, null);
     RMDelegationTokenIdentifier dtId1 = token1.decodeIdentifier();

+ 6 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

@@ -59,7 +59,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -153,7 +152,8 @@ public class TestContainerManagerSecurity {
     final ContainerId containerID = allocatedContainer.getId();
     UserGroupInformation authenticatedUser = UserGroupInformation
         .createRemoteUser(containerID.toString());
-    ContainerToken containerToken = allocatedContainer.getContainerToken();
+    org.apache.hadoop.yarn.api.records.Token containerToken =
+        allocatedContainer.getContainerToken();
     Token<ContainerTokenIdentifier> token = new Token<ContainerTokenIdentifier>(
         containerToken.getIdentifier().array(), containerToken.getPassword()
             .array(), new Text(containerToken.getKind()), new Text(
@@ -212,7 +212,8 @@ public class TestContainerManagerSecurity {
 
     // Now talk to the NM for launching the container with modified resource
 
-    ContainerToken containerToken = allocatedContainer.getContainerToken();
+    org.apache.hadoop.yarn.api.records.Token containerToken =
+        allocatedContainer.getContainerToken();
     ContainerTokenIdentifier originalContainerTokenId =
         BuilderUtils.newContainerTokenIdentifier(containerToken);
 
@@ -327,7 +328,8 @@ public class TestContainerManagerSecurity {
     // Now talk to the NM for launching the container with modified containerID
     final ContainerId containerID = allocatedContainer.getId();
 
-    ContainerToken containerToken = allocatedContainer.getContainerToken();
+    org.apache.hadoop.yarn.api.records.Token containerToken =
+        allocatedContainer.getContainerToken();
     final ContainerTokenIdentifier tokenId =
         BuilderUtils.newContainerTokenIdentifier(containerToken);