Browse Source

svn merge -c -1476006 . for reverting the previous trunk merge.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1476008 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 years ago
parent
commit
43ac0739ce
57 changed files with 530 additions and 2502 deletions
  1. 0 3
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 6 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
  3. 111 159
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  4. 4 66
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  5. 1 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
  6. 4 20
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
  7. 1 37
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  8. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java
  9. 0 20
      hadoop-mapreduce-project/CHANGES.txt
  10. 22 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  11. 6 17
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java
  12. 10 73
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
  13. 17 27
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java
  14. 0 86
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReaderWrapper.java
  15. 0 66
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineSequenceFileInputFormat.java
  16. 0 68
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineTextInputFormat.java
  17. 3 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java
  18. 7 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java
  19. 0 108
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileRecordReaderWrapper.java
  20. 0 64
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineSequenceFileInputFormat.java
  21. 0 65
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineTextInputFormat.java
  22. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java
  23. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
  24. 0 170
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java
  25. 0 250
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java
  26. 0 334
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapperReducerCleanup.java
  27. 0 186
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java
  28. 0 267
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java
  29. 0 16
      hadoop-yarn-project/CHANGES.txt
  30. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
  31. 0 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
  32. 0 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
  33. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  34. 22 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java
  35. 3 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
  36. 4 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
  37. 1 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java
  38. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java
  39. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
  40. 10 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
  41. 0 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java
  42. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  43. 26 45
      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
  44. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
  45. 6 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
  46. 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/FSQueue.java
  47. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
  48. 27 56
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
  49. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  50. 150 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Resources.java
  51. 10 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
  52. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
  53. 29 58
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
  54. 0 43
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResources.java
  55. 0 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  56. 24 31
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java
  57. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java

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

@@ -559,9 +559,6 @@ Release 2.0.5-beta - UNRELEASED
     HADOOP-9450. HADOOP_USER_CLASSPATH_FIRST is not honored; CLASSPATH
     HADOOP-9450. HADOOP_USER_CLASSPATH_FIRST is not honored; CLASSPATH
     is PREpended instead of APpended. (Chris Nauroth and harsh via harsh)
     is PREpended instead of APpended. (Chris Nauroth and harsh via harsh)
 
 
-    HADOOP-9503. Remove sleep between IPC client connect timeouts.
-    (Varun Sharma via szetszwo)
-
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-9150. Avoid unnecessary DNS resolution attempts for logical URIs
     HADOOP-9150. Avoid unnecessary DNS resolution attempts for logical URIs

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -706,6 +706,12 @@ public class Client {
       if (curRetries >= maxRetries) {
       if (curRetries >= maxRetries) {
         throw ioe;
         throw ioe;
       }
       }
+
+      // otherwise back off and retry
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException ignored) {}
+      
       LOG.info("Retrying connect to server: " + server + ". Already tried "
       LOG.info("Retrying connect to server: " + server + ". Already tried "
           + curRetries + " time(s); maxRetries=" + maxRetries);
           + curRetries + " time(s); maxRetries=" + maxRetries);
     }
     }

+ 111 - 159
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -4,10 +4,10 @@ Trunk (Unreleased)
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
 
 
-    HDFS-4053. Increase the default block size. (eli)
-
     HDFS-3034. Remove the deprecated DFSOutputStream.sync() method.  (szetszwo)
     HDFS-3034. Remove the deprecated DFSOutputStream.sync() method.  (szetszwo)
 
 
+    HDFS-4434. Provide a mapping from INodeId to INode. (suresh)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
@@ -15,9 +15,15 @@ Trunk (Unreleased)
     HDFS-3601. Add BlockPlacementPolicyWithNodeGroup to support block placement
     HDFS-3601. Add BlockPlacementPolicyWithNodeGroup to support block placement
     with 4-layer network topology.  (Junping Du via szetszwo)
     with 4-layer network topology.  (Junping Du via szetszwo)
 
 
+    HDFS-3077. Implement QuorumJournalManager, a distributed mechanism for
+    reliably storing HDFS edit logs. See dedicated section below for breakdown
+    of subtasks.
+
     HDFS-3495. Update Balancer to support new NetworkTopology with NodeGroup.
     HDFS-3495. Update Balancer to support new NetworkTopology with NodeGroup.
     (Junping Du via szetszwo)
     (Junping Du via szetszwo)
 
 
+    HDFS-4296. Reserve layout version for release 1.2.0. (suresh)
+
     HADOOP-8562. Enhancements to support Hadoop on Windows Server and Windows
     HADOOP-8562. Enhancements to support Hadoop on Windows Server and Windows
     Azure environments. (See breakdown of tasks below for subtasks and
     Azure environments. (See breakdown of tasks below for subtasks and
     contributors)
     contributors)
@@ -94,17 +100,38 @@ Trunk (Unreleased)
     HDFS-3476. Correct the default used in TestDFSClientRetries.busyTest()
     HDFS-3476. Correct the default used in TestDFSClientRetries.busyTest()
     after HDFS-3462 (harsh)
     after HDFS-3462 (harsh)
 
 
+    HDFS-3040. TestMulitipleNNDataBlockScanner is misspelled. (Madhukara Phatak
+    via atm)
+
     HDFS-3478. Test quotas with Long.Max_Value. (Sujay Rau via eli)
     HDFS-3478. Test quotas with Long.Max_Value. (Sujay Rau via eli)
 
 
     HDFS-3498. Support replica removal in BlockPlacementPolicy and make
     HDFS-3498. Support replica removal in BlockPlacementPolicy and make
     BlockPlacementPolicyDefault extensible for reusing code in subclasses.
     BlockPlacementPolicyDefault extensible for reusing code in subclasses.
     (Junping Du via szetszwo)
     (Junping Du via szetszwo)
 
 
+    HDFS-3510.  Editlog pre-allocation is performed prior to writing edits
+    to avoid partial edits case disk out of space.(Colin McCabe via suresh)
+
     HDFS-3630 Modify TestPersistBlocks to use both flush and hflush  (sanjay)
     HDFS-3630 Modify TestPersistBlocks to use both flush and hflush  (sanjay)
 
 
     HDFS-3768. Exception in TestJettyHelper is incorrect. 
     HDFS-3768. Exception in TestJettyHelper is incorrect. 
     (Eli Reisman via jghoman)
     (Eli Reisman via jghoman)
 
 
+    HDFS-3723. Add support -h, -help to all the commands. (Jing Zhao via
+    suresh)
+
+    HDFS-3803. Change BlockPoolSliceScanner chatty INFO log to DEBUG.
+    (Andrew Purtell via suresh)
+
+    HDFS-3817. Avoid printing SafeModeException stack trace.
+    (Brandon Li via suresh)
+
+    HDFS-3819. Should check whether invalidate work percentage default value is 
+    not greater than 1.0f. (Jing Zhao via jitendra)
+
+    HDFS-3844. Add @Override and remove {@inheritdoc} and unnecessary
+    imports. (Jing Zhao via suresh)
+
     HDFS-3851. DFSOutputStream class code cleanup. (Jing Zhao via suresh)
     HDFS-3851. DFSOutputStream class code cleanup. (Jing Zhao via suresh)
 
 
     HDFS-2580. NameNode#main(...) can make use of GenericOptionsParser. (harsh)
     HDFS-2580. NameNode#main(...) can make use of GenericOptionsParser. (harsh)
@@ -125,14 +152,40 @@ Trunk (Unreleased)
     HDFS-4052. BlockManager#invalidateWork should print log outside the lock.
     HDFS-4052. BlockManager#invalidateWork should print log outside the lock.
     (Jing Zhao via suresh)
     (Jing Zhao via suresh)
 
 
+    HDFS-4124. Refactor INodeDirectory#getExistingPathINodes() to enable 
+    returning more than INode array. (Jing Zhao via suresh)
+
+    HDFS-4129. Add utility methods to dump NameNode in memory tree for 
+    testing. (szetszwo via suresh)
+
+    HDFS-4151. Change the methods in FSDirectory to pass INodesInPath instead
+    of INode[] as a parameter. (szetszwo)
+
+    HDFS-4152. Add a new class BlocksMapUpdateInfo for the parameter in
+    INode.collectSubtreeBlocksAndClear(..). (Jing Zhao via szetszwo)
+
+    HDFS-4206. Change the fields in INode and its subclasses to private.
+    (szetszwo)
+
+    HDFS-4215. Remove locking from addToParent(..) since it is used in image
+    loading, and add INode.isFile().  (szetszwo)
+
     HDFS-4200. Reduce the size of synchronized sections in PacketResponder.
     HDFS-4200. Reduce the size of synchronized sections in PacketResponder.
     (suresh)
     (suresh)
 
 
+    HDFS-4209. Clean up the addNode/addChild/addChildNoQuotaCheck methods in
+    FSDirectory and INodeDirectory. (szetszwo)
+
     HDFS-3358. Specify explicitly that the NN UI status total is talking
     HDFS-3358. Specify explicitly that the NN UI status total is talking
     of persistent objects on heap. (harsh)
     of persistent objects on heap. (harsh)
 
 
     HDFS-4234. Use generic code for choosing datanode in Balancer.  (szetszwo)
     HDFS-4234. Use generic code for choosing datanode in Balancer.  (szetszwo)
 
 
+    HDFS-4334. Add a unique id to INode.  (Brandon Li via szetszwo)
+
+    HDFS-4346. Add SequentialNumber as a base class for INodeId and
+    GenerationStamp.  (szetszwo)
+
     HDFS-4633 TestDFSClientExcludedNodes fails sporadically if excluded nodes
     HDFS-4633 TestDFSClientExcludedNodes fails sporadically if excluded nodes
     cache expires too quickly  (Chris Nauroth via Sanjay)
     cache expires too quickly  (Chris Nauroth via Sanjay)
 
 
@@ -161,9 +214,14 @@ Trunk (Unreleased)
     HDFS-2776. Missing interface annotation on JournalSet. 
     HDFS-2776. Missing interface annotation on JournalSet. 
     (Brandon Li via jitendra)
     (Brandon Li via jitendra)
 
 
+    HDFS-2908. Add apache license header for StorageReport.java. (Brandon Li
+    via jitendra)
+
     HDFS-3037. TestMulitipleNNDataBlockScanner#testBlockScannerAfterRestart is
     HDFS-3037. TestMulitipleNNDataBlockScanner#testBlockScannerAfterRestart is
     racy. (atm)
     racy. (atm)
 
 
+    HDFS-2966. TestNameNodeMetrics tests can fail under load. (stevel)
+
     HDFS-3116. Typo in fetchdt error message. (AOE Takashi via atm)
     HDFS-3116. Typo in fetchdt error message. (AOE Takashi via atm)
 
 
     HDFS-3126. Journal stream from Namenode to BackupNode needs to have 
     HDFS-3126. Journal stream from Namenode to BackupNode needs to have 
@@ -183,6 +241,8 @@ Trunk (Unreleased)
     HDFS-3163. TestHDFSCLI.testAll fails if the user name is not all lowercase.
     HDFS-3163. TestHDFSCLI.testAll fails if the user name is not all lowercase.
     (Brandon Li via atm)
     (Brandon Li via atm)
 
 
+    HDFS-3368. Missing blocks due to bad DataNodes coming up and down. (shv)
+
     HDFS-3462. TestDFSClientRetries.busyTest() should restore default
     HDFS-3462. TestDFSClientRetries.busyTest() should restore default
     xceiver count in the config. (Madhukara Phatak via harsh)
     xceiver count in the config. (Madhukara Phatak via harsh)
 
 
@@ -190,6 +250,8 @@ Trunk (Unreleased)
 
 
     HDFS-3549. Fix dist tar build fails in hadoop-hdfs-raid project. (Jason Lowe via daryn)
     HDFS-3549. Fix dist tar build fails in hadoop-hdfs-raid project. (Jason Lowe via daryn)
 
 
+    HDFS-3541. Deadlock between recovery, xceiver and packet responder (Vinay via umamahesh)
+
     HDFS-3482. hdfs balancer throws ArrayIndexOutOfBoundsException 
     HDFS-3482. hdfs balancer throws ArrayIndexOutOfBoundsException 
     if option is specified without values. ( Madhukara Phatak via umamahesh) 
     if option is specified without values. ( Madhukara Phatak via umamahesh) 
 
 
@@ -222,9 +284,6 @@ Trunk (Unreleased)
     HDFS-4105. The SPNEGO user for secondary namenode should use the web 
     HDFS-4105. The SPNEGO user for secondary namenode should use the web 
     keytab. (Arpit Gupta via jitendra)
     keytab. (Arpit Gupta via jitendra)
 
 
-    HDFS-4003. test-patch should build the common native libs before
-    running hdfs tests. (Colin Patrick McCabe via eli)
-
     HDFS-4240. For nodegroup-aware block placement, when a node is excluded,
     HDFS-4240. For nodegroup-aware block placement, when a node is excluded,
     the nodes in the same nodegroup should also be excluded.  (Junping Du
     the nodes in the same nodegroup should also be excluded.  (Junping Du
     via szetszwo)
     via szetszwo)
@@ -232,6 +291,9 @@ Trunk (Unreleased)
     HDFS-4260 Fix HDFS tests to set test dir to a valid HDFS path as opposed
     HDFS-4260 Fix HDFS tests to set test dir to a valid HDFS path as opposed
     to the local build path (Chri Nauroth via Sanjay)
     to the local build path (Chri Nauroth via Sanjay)
 
 
+    HDFS-4269. Datanode rejects all datanode registrations from localhost
+    in single-node developer setup on Windows. (Chris Nauroth via suresh)
+
     HADOOP-8957 HDFS tests for AbstractFileSystem#IsValidName should be overridden for
     HADOOP-8957 HDFS tests for AbstractFileSystem#IsValidName should be overridden for
     embedded file systems like ViewFs (Chris Nauroth via Sanjay Radia)
     embedded file systems like ViewFs (Chris Nauroth via Sanjay Radia)
 
 
@@ -249,19 +311,25 @@ Trunk (Unreleased)
 
 
     HDFS-4382. Fix typo MAX_NOT_CHANGED_INTERATIONS. (Ted Yu via suresh)
     HDFS-4382. Fix typo MAX_NOT_CHANGED_INTERATIONS. (Ted Yu via suresh)
 
 
+    HDFS-4340. Update addBlock() to inculde inode id as additional argument.
+    (Brandon Li via suresh)
+
+    HDFS-4502. JsonUtil.toFileStatus(..) should check if the fileId property
+    exists.  (Brandon Li via suresh)
+
     HDFS-4391. TestDataTransferKeepalive fails when tests are executed in a
     HDFS-4391. TestDataTransferKeepalive fails when tests are executed in a
     certain order. (Andrew Wang via atm)
     certain order. (Andrew Wang via atm)
 
 
     HDFS-4586. TestDataDirs.testGetDataDirsFromURIs fails with all directories
     HDFS-4586. TestDataDirs.testGetDataDirsFromURIs fails with all directories
     in dfs.datanode.data.dir are invalid. (Ivan Mitic via atm)
     in dfs.datanode.data.dir are invalid. (Ivan Mitic via atm)
 
 
+    HDFS-4646. createNNProxyWithClientProtocol ignores configured timeout
+    value (Jagane Sundar via cos)
+
     HDFS-4732. Fix TestDFSUpgradeFromImage which fails on Windows due to
     HDFS-4732. Fix TestDFSUpgradeFromImage which fails on Windows due to
     failure to unpack old image tarball that contains hard links.
     failure to unpack old image tarball that contains hard links.
     (Chris Nauroth via szetszwo)
     (Chris Nauroth via szetszwo)
 
 
-    HDFS-4757. Update FSDirectory#inodeMap when replacing an INodeDirectory
-    while setting quota.  (Jing Zhao via szetszwo)
-
   BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
   BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
 
 
     HDFS-4145. Merge hdfs cmd line scripts from branch-1-win. (David Lao,
     HDFS-4145. Merge hdfs cmd line scripts from branch-1-win. (David Lao,
@@ -299,12 +367,6 @@ Trunk (Unreleased)
     HDFS-4725. Fix HDFS file handle leaks in FSEditLog, NameNode,
     HDFS-4725. Fix HDFS file handle leaks in FSEditLog, NameNode,
     OfflineEditsBinaryLoader and some tests.  (Chris Nauroth via szetszwo)
     OfflineEditsBinaryLoader and some tests.  (Chris Nauroth via szetszwo)
 
 
-    HDFS-4615. Fix TestDFSShell failures on Windows.  (Arpit Agarwal
-    via szetszwo)
-
-    HDFS-4584. Skip TestNNWithQJM.testNewNamenodeTakesOverWriter() on Windows.
-    (Arpit Agarwal via szetszwo)
-
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
 
 
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
@@ -365,26 +427,11 @@ Release 2.0.5-beta - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
 
 
-    HDFS-4434. Provide a mapping from INodeId to INode. (suresh)
-
   NEW FEATURES
   NEW FEATURES
 
 
     HDFS-1804. Add a new block-volume device choosing policy that looks at
     HDFS-1804. Add a new block-volume device choosing policy that looks at
     free space. (atm)
     free space. (atm)
 
 
-    HDFS-4296. Reserve layout version for release 1.2.0. (suresh)
-
-    HDFS-4334. Add a unique id to INode.  (Brandon Li via szetszwo)
-
-    HDFS-4339. Persist inode id in fsimage and editlog. (Brandon Li via
-    suresh)
-
-    HDFS-4340. Update addBlock() to inculde inode id as additional argument.
-    (Brandon Li via suresh)
-
-    HDFS-4502. JsonUtil.toFileStatus(..) should check if the fileId property
-    exists.  (Brandon Li via suresh)
-
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-4222. NN is unresponsive and loses heartbeats from DNs when 
     HDFS-4222. NN is unresponsive and loses heartbeats from DNs when 
@@ -426,37 +473,6 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4693. Some test cases in TestCheckpoint do not clean up after 
     HDFS-4693. Some test cases in TestCheckpoint do not clean up after 
     themselves. (Arpit Agarwal, suresh via suresh)
     themselves. (Arpit Agarwal, suresh via suresh)
 
 
-    HDFS-3817. Avoid printing SafeModeException stack trace.
-    (Brandon Li via suresh)
-
-    HDFS-4124. Refactor INodeDirectory#getExistingPathINodes() to enable 
-    returning more than INode array. (Jing Zhao via suresh)
-
-    HDFS-4151. Change the methods in FSDirectory to pass INodesInPath instead
-    of INode[] as a parameter. (szetszwo)
-
-    HDFS-4129. Add utility methods to dump NameNode in memory tree for 
-    testing. (szetszwo via suresh)
-
-    HDFS-4152. Add a new class BlocksMapUpdateInfo for the parameter in
-    INode.collectSubtreeBlocksAndClear(..). (Jing Zhao via szetszwo)
-
-    HDFS-4206. Change the fields in INode and its subclasses to private.
-    (szetszwo)
-
-    HDFS-4215. Remove locking from addToParent(..) since it is used in image
-    loading, and add INode.isFile().  (szetszwo)
-
-    HDFS-4243. When replacing an INodeDirectory, the parent pointers of the
-    children of the child have to be updated to the new child.  (Jing Zhao
-    via szetszwo)
-
-    HDFS-4209. Clean up the addNode/addChild/addChildNoQuotaCheck methods in
-    FSDirectory and INodeDirectory. (szetszwo)
-
-    HDFS-4346. Add SequentialNumber as a base class for INodeId and
-    GenerationStamp.  (szetszwo)
-
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -474,9 +490,6 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4482. ReplicationMonitor thread can exit with NPE due to the race 
     HDFS-4482. ReplicationMonitor thread can exit with NPE due to the race 
     between delete and replication of same file. (umamahesh)
     between delete and replication of same file. (umamahesh)
 
 
-    HDFS-4269. Datanode rejects all datanode registrations from localhost
-    in single-node developer setup on Windows. (Chris Nauroth via suresh)
-
     HDFS-4235. When outputting XML, OfflineEditsViewer can't handle some edits
     HDFS-4235. When outputting XML, OfflineEditsViewer can't handle some edits
     containing non-ASCII strings. (Colin Patrick McCabe via atm)
     containing non-ASCII strings. (Colin Patrick McCabe via atm)
 
 
@@ -533,6 +546,12 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4609. TestAuditLogs should release log handles between tests. 
     HDFS-4609. TestAuditLogs should release log handles between tests. 
     (Ivan Mitic via szetszwo)
     (Ivan Mitic via szetszwo)
 
 
+    HDFS-4615. Fix TestDFSShell failures on Windows.  (Arpit Agarwal
+    via szetszwo)
+
+    HDFS-4584. Skip TestNNWithQJM.testNewNamenodeTakesOverWriter() on Windows.
+    (Arpit Agarwal via szetszwo)
+
     HDFS-4598. Fix the default value of ConcatSourcesParam and the WebHDFS doc.
     HDFS-4598. Fix the default value of ConcatSourcesParam and the WebHDFS doc.
     (szetszwo)
     (szetszwo)
 
 
@@ -544,9 +563,6 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4658. Standby NN will log that it has received a block report "after
     HDFS-4658. Standby NN will log that it has received a block report "after
     becoming active" (atm)
     becoming active" (atm)
 
 
-    HDFS-4646. createNNProxyWithClientProtocol ignores configured timeout
-    value (Jagane Sundar via cos)
-
     HDFS-3981. Fix handling of FSN lock in getBlockLocations. (Xiaobo Peng
     HDFS-3981. Fix handling of FSN lock in getBlockLocations. (Xiaobo Peng
     and todd via todd)
     and todd via todd)
 
 
@@ -566,12 +582,6 @@ Release 2.0.5-beta - UNRELEASED
 
 
     HDFS-4737. JVM path embedded in fuse binaries. (Sean Mackrory via atm)
     HDFS-4737. JVM path embedded in fuse binaries. (Sean Mackrory via atm)
 
 
-    HDFS-4739. NN can miscalculate the number of extra edit log segments to
-    retain. (atm)
-
-    HDFS-4745. TestDataTransferKeepalive#testSlowReader has race condition that
-    causes sporadic failure. (Chris Nauroth via suresh)
-
 Release 2.0.4-alpha - UNRELEASED
 Release 2.0.4-alpha - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -624,10 +634,6 @@ Release 2.0.3-alpha - 2013-02-06
     HDFS-4213. Add an API to hsync for updating the last block length at the
     HDFS-4213. Add an API to hsync for updating the last block length at the
     namenode. (Jing Zhao via szetszwo)
     namenode. (Jing Zhao via szetszwo)
 
 
-    HDFS-3077. Implement QuorumJournalManager, a distributed mechanism for
-    reliably storing HDFS edit logs. See dedicated section below for breakdown
-    of subtasks.
-
   IMPROVEMENTS
   IMPROVEMENTS
   
   
     HDFS-3925. Prettify PipelineAck#toString() for printing to a log
     HDFS-3925. Prettify PipelineAck#toString() for printing to a log
@@ -685,6 +691,8 @@ Release 2.0.3-alpha - 2013-02-06
     HDFS-4074. Remove the unused default constructor from INode.  (Brandon Li
     HDFS-4074. Remove the unused default constructor from INode.  (Brandon Li
     via szetszwo)
     via szetszwo)
 
 
+    HDFS-4053. Increase the default block size. (eli)
+
     HDFS-4088. Remove "throws QuotaExceededException" from an
     HDFS-4088. Remove "throws QuotaExceededException" from an
     INodeDirectoryWithQuota constructor. (szetszwo)
     INodeDirectoryWithQuota constructor. (szetszwo)
 
 
@@ -700,17 +708,12 @@ Release 2.0.3-alpha - 2013-02-06
     HDFS-4121. Add namespace declarations in hdfs .proto files for languages 
     HDFS-4121. Add namespace declarations in hdfs .proto files for languages 
     other than java. (Binglin Chang via suresh)
     other than java. (Binglin Chang via suresh)
 
 
-    HDFS-3573. Supply NamespaceInfo when instantiating JournalManagers. 
-    (todd and ivank via umamahesh)
+    HDFS-3573. Supply NamespaceInfo when instantiating JournalManagers (todd)
 
 
-    HDFS-3695. Genericize format() to non-file JournalManagers. 
-    (todd via umamahesh)
+    HDFS-3695. Genericize format() to non-file JournalManagers. (todd)
 
 
-    HDFS-3789. JournalManager#format() should be able to throw IOException. 
-    (Ivan Kelly via umamahesh)
-
-    HDFS-3809. Make BKJM use protobufs for all serialization with ZK. 
-    (Ivan Kelly via umamhesh)
+    HDFS-3789. JournalManager#format() should be able to throw IOException
+    (Ivan Kelly via todd)
 
 
     HDFS-3916. libwebhdfs testing code cleanup. (Jing Zhao via suresh)
     HDFS-3916. libwebhdfs testing code cleanup. (Jing Zhao via suresh)
 
 
@@ -768,7 +771,7 @@ Release 2.0.3-alpha - 2013-02-06
 
 
     HDFS-4031. Update findbugsExcludeFile.xml to include findbugs 2
     HDFS-4031. Update findbugsExcludeFile.xml to include findbugs 2
     exclusions. (eli)
     exclusions. (eli)
-
+    
     HDFS-4033. Miscellaneous findbugs 2 fixes. (eli)
     HDFS-4033. Miscellaneous findbugs 2 fixes. (eli)
 
 
     HDFS-4034. Remove redundant null checks. (eli)
     HDFS-4034. Remove redundant null checks. (eli)
@@ -845,6 +848,9 @@ Release 2.0.3-alpha - 2013-02-06
     HDFS-4006. TestCheckpoint#testSecondaryHasVeryOutOfDateImage
     HDFS-4006. TestCheckpoint#testSecondaryHasVeryOutOfDateImage
     occasionally fails due to unexpected exit. (todd via eli)
     occasionally fails due to unexpected exit. (todd via eli)
 
 
+    HDFS-4003. test-patch should build the common native libs before
+    running hdfs tests. (Colin Patrick McCabe via eli)
+
     HDFS-4018. testMiniDFSClusterWithMultipleNN is missing some
     HDFS-4018. testMiniDFSClusterWithMultipleNN is missing some
     cluster cleanup. (eli)
     cluster cleanup. (eli)
 
 
@@ -892,6 +898,9 @@ Release 2.0.3-alpha - 2013-02-06
     HADOOP-8994. TestDFSShell creates file named "noFileHere", making further
     HADOOP-8994. TestDFSShell creates file named "noFileHere", making further
     tests hard to understand (Andy Isaacson via daryn)
     tests hard to understand (Andy Isaacson via daryn)
 
 
+    HDFS-3809. Make BKJM use protobufs for all serialization with ZK.
+    (Ivan Kelly via umamahesh)
+
     HDFS-3804.  TestHftpFileSystem fails intermittently with JDK7
     HDFS-3804.  TestHftpFileSystem fails intermittently with JDK7
     (Trevor Robinson via daryn)
     (Trevor Robinson via daryn)
 
 
@@ -903,14 +912,14 @@ Release 2.0.3-alpha - 2013-02-06
     HDFS-3979. For hsync, datanode should wait for the local sync to complete
     HDFS-3979. For hsync, datanode should wait for the local sync to complete
     before sending ack. (Lars Hofhansl via szetszwo)
     before sending ack. (Lars Hofhansl via szetszwo)
 
 
-    HDFS-3810. Implement format() for BKJM (Ivan Kelly via umamahesh)
-
     HDFS-3625. Fix TestBackupNode by properly initializing edit log during
     HDFS-3625. Fix TestBackupNode by properly initializing edit log during
     startup. (Junping Du via todd)
     startup. (Junping Du via todd)
 
 
     HDFS-4138. BackupNode startup fails due to uninitialized edit log.
     HDFS-4138. BackupNode startup fails due to uninitialized edit log.
     (Kihwal Lee via shv)
     (Kihwal Lee via shv)
 
 
+    HDFS-3810. Implement format() for BKJM (Ivan Kelly via umamahesh)
+
     HDFS-4162. Some malformed and unquoted HTML strings are returned from 
     HDFS-4162. Some malformed and unquoted HTML strings are returned from 
     datanode web ui. (Darek Dagit via suresh)
     datanode web ui. (Darek Dagit via suresh)
 
 
@@ -937,9 +946,6 @@ Release 2.0.3-alpha - 2013-02-06
     HDFS-3507. DFS#isInSafeMode needs to execute only on Active NameNode.
     HDFS-3507. DFS#isInSafeMode needs to execute only on Active NameNode.
     (Vinay via atm)
     (Vinay via atm)
 
 
-    HDFS-4105. The SPNEGO user for secondary namenode should use the web
-    keytab. (Arpit Gupta via jitendra)
-
     HDFS-4156. Seeking to a negative position should throw an IOE.
     HDFS-4156. Seeking to a negative position should throw an IOE.
     (Eli Reisman via eli)
     (Eli Reisman via eli)
 
 
@@ -959,6 +965,10 @@ Release 2.0.3-alpha - 2013-02-06
 
 
     HDFS-4231. BackupNode: Introduce BackupState. (shv)
     HDFS-4231. BackupNode: Introduce BackupState. (shv)
 
 
+    HDFS-4243. When replacing an INodeDirectory, the parent pointers of the
+    children of the child have to be updated to the new child.  (Jing Zhao
+    via szetszwo)
+
     HDFS-4238. Standby namenode should not do purging of shared
     HDFS-4238. Standby namenode should not do purging of shared
     storage edits. (todd)
     storage edits. (todd)
 
 
@@ -1162,9 +1172,6 @@ Release 2.0.3-alpha - 2013-02-06
     HDFS-4351. In BlockPlacementPolicyDefault.chooseTarget(..), numOfReplicas
     HDFS-4351. In BlockPlacementPolicyDefault.chooseTarget(..), numOfReplicas
     needs to be updated when avoiding stale nodes.  (Andrew Wang via szetszwo)
     needs to be updated when avoiding stale nodes.  (Andrew Wang via szetszwo)
 
 
-    HDFS-2908. Add apache license header for StorageReport.java. (Brandon Li
-    via tgraves)
-
     HDFS-4399. Fix RAT warnings by excluding images sub-dir in docs. (Thomas
     HDFS-4399. Fix RAT warnings by excluding images sub-dir in docs. (Thomas
     Graves via acmurthy) 
     Graves via acmurthy) 
 
 
@@ -1202,10 +1209,8 @@ Release 2.0.2-alpha - 2012-09-07
     configured timeout and are selected as the last location to read from.
     configured timeout and are selected as the last location to read from.
     (Jing Zhao via suresh)
     (Jing Zhao via suresh)
     
     
-  IMPROVEMENTS
 
 
-    HDFS-3040. TestMulitipleNNDataBlockScanner is misspelled. (Madhukara Phatak
-    via atm)
+  IMPROVEMENTS
 
 
     HDFS-3390. DFSAdmin should print full stack traces of errors when DEBUG
     HDFS-3390. DFSAdmin should print full stack traces of errors when DEBUG
     logging is enabled. (atm)
     logging is enabled. (atm)
@@ -1380,12 +1385,6 @@ Release 2.0.2-alpha - 2012-09-07
     HDFS-3765. namenode -initializeSharedEdits should be able to initialize
     HDFS-3765. namenode -initializeSharedEdits should be able to initialize
     all shared storages. (Vinay and todd via todd)
     all shared storages. (Vinay and todd via todd)
 
 
-    HDFS-3723. Add support -h, -help to all the commands. (Jing Zhao via
-    suresh)
-
-    HDFS-3803. Change BlockPoolSliceScanner chatty INFO log to DEBUG.
-    (Andrew Purtell via suresh)
-
     HDFS-3802. StartupOption.name in HdfsServerConstants should be final.
     HDFS-3802. StartupOption.name in HdfsServerConstants should be final.
     (Jing Zhao via szetszwo)
     (Jing Zhao via szetszwo)
 
 
@@ -1400,20 +1399,13 @@ Release 2.0.2-alpha - 2012-09-07
     HDFS-2727. libhdfs should get the default block size from the server.
     HDFS-2727. libhdfs should get the default block size from the server.
     (Colin Patrick McCabe via eli)
     (Colin Patrick McCabe via eli)
 
 
-    HDFS-3832. Remove protocol methods related to DistributedUpgrade. (suresh)
+    HDFS-2686. Remove DistributedUpgrade related code. (suresh)
 
 
-    HDFS-3819. Should check whether invalidate work percentage default value is
-    not greater than 1.0f. (Jing Zhao via jitendra)
+    HDFS-3832. Remove protocol methods related to DistributedUpgrade. (suresh)
 
 
     HDFS-3177. Update DFSClient and DataXceiver to handle different checkum
     HDFS-3177. Update DFSClient and DataXceiver to handle different checkum
     types in file checksum computation.  (Kihwal Lee via szetszwo)
     types in file checksum computation.  (Kihwal Lee via szetszwo)
 
 
-    HDFS-3844. Add @Override and remove {@inheritdoc} and unnecessary
-    imports. (Jing Zhao via suresh)
-
-    HDFS-3853. Port MiniDFSCluster enableManagedDfsDirsRedundancy
-    option to branch-2. (Colin Patrick McCabe via eli)
-
     HDFS-3871. Change NameNodeProxies to use RetryUtils.  (Arun C Murthy
     HDFS-3871. Change NameNodeProxies to use RetryUtils.  (Arun C Murthy
     via szetszwo)
     via szetszwo)
 
 
@@ -1424,9 +1416,6 @@ Release 2.0.2-alpha - 2012-09-07
 
 
     HDFS-3907. Allow multiple users for local block readers. (eli)
     HDFS-3907. Allow multiple users for local block readers. (eli)
 
 
-    HDFS-3510. Editlog pre-allocation is performed prior to writing edits
-    to avoid partial edits case disk out of space. (Colin McCabe via todd)
-
     HDFS-3910. DFSTestUtil#waitReplication should timeout. (eli)
     HDFS-3910. DFSTestUtil#waitReplication should timeout. (eli)
     
     
     HDFS-3920. libwebdhfs string processing and using strerror consistently
     HDFS-3920. libwebdhfs string processing and using strerror consistently
@@ -1486,9 +1475,6 @@ Release 2.0.2-alpha - 2012-09-07
 
 
     HDFS-3444. hdfs groups command doesn't work with security enabled. (atm)
     HDFS-3444. hdfs groups command doesn't work with security enabled. (atm)
 
 
-    HDFS-2717. BookKeeper Journal output stream doesn't check addComplete rc.
-    (Ivan Kelly via umamahesh)
-
     HDFS-3415. Make sure all layout versions are the same for all storage
     HDFS-3415. Make sure all layout versions are the same for all storage
     directories in the Namenode.  (Brandon Li via szetszwo)
     directories in the Namenode.  (Brandon Li via szetszwo)
 
 
@@ -1499,28 +1485,6 @@ Release 2.0.2-alpha - 2012-09-07
     HDFS-3460. HttpFS proxyuser validation with Kerberos ON uses full 
     HDFS-3460. HttpFS proxyuser validation with Kerberos ON uses full 
     principal name. (tucu)
     principal name. (tucu)
 
 
-    HDFS-3058. HA: Bring BookKeeperJournalManager up to date with HA changes.
-    (Ivan Kelly via umamahesh)
-
-    HDFS-3368. Missing blocks due to bad DataNodes coming up and down. (shv)
-
-    HDFS-3452. BKJM:Switch from standby to active fails and NN gets shut down
-    due to delay in clearing of lock. (umamahesh)
-
-    HDFS-3398. Client will not retry when primaryDN is down once it's just got pipeline.
-    (Amith D K via umamahesh)
-
-    HDFS-3474. Cleanup Exception handling in BookKeeper journal manager.
-    (Ivan Kelly via umamahesh)
-
-    HDFS-3468. Make BKJM-ZK session timeout configurable. (umamahesh)
-
-    HDFS-3423. BKJM: NN startup is failing, when tries to recoverUnfinalizedSegments()
-    a bad inProgress_ ZNodes. (Ivan Kelly and Uma via umamahesh)
-
-    HDFS-3441. Race condition between rolling logs at active NN and purging at standby.
-    (Rakesh R via umamahesh)
-
     HDFS-3484. hdfs fsck doesn't work if NN HTTP address is set to
     HDFS-3484. hdfs fsck doesn't work if NN HTTP address is set to
     0.0.0.0 even if NN RPC address is configured. (atm via eli)
     0.0.0.0 even if NN RPC address is configured. (atm via eli)
 
 
@@ -1533,9 +1497,6 @@ Release 2.0.2-alpha - 2012-09-07
     HDFS-3442. Incorrect count for Missing Replicas in FSCK report. (Andrew
     HDFS-3442. Incorrect count for Missing Replicas in FSCK report. (Andrew
     Wang via atm)
     Wang via atm)
 
 
-    HDFS-2025. Go Back to File View link is not working in tail.jsp.
-    (Ashish and Sravan via umamahesh)
-
     HDFS-3501. Checkpointing with security enabled will stop working
     HDFS-3501. Checkpointing with security enabled will stop working
     after ticket lifetime expires. (atm via eli)
     after ticket lifetime expires. (atm via eli)
 
 
@@ -1568,7 +1529,7 @@ Release 2.0.2-alpha - 2012-09-07
 
 
     HDFS-3522. If a namenode is in safemode, it should throw SafeModeException
     HDFS-3522. If a namenode is in safemode, it should throw SafeModeException
     when getBlockLocations has zero locations.  (Brandon Li via szetszwo)
     when getBlockLocations has zero locations.  (Brandon Li via szetszwo)
-
+    
     HDFS-3408. BKJM : Namenode format fails, if there is no BK root. (Rakesh R via umamahesh)
     HDFS-3408. BKJM : Namenode format fails, if there is no BK root. (Rakesh R via umamahesh)
 
 
     HDFS-3389. Document the BKJM usage in Namenode HA. (umamahesh and Ivan Kelly via umamahesh)
     HDFS-3389. Document the BKJM usage in Namenode HA. (umamahesh and Ivan Kelly via umamahesh)
@@ -1601,9 +1562,6 @@ Release 2.0.2-alpha - 2012-09-07
     HDFS-3581. FSPermissionChecker#checkPermission sticky bit check
     HDFS-3581. FSPermissionChecker#checkPermission sticky bit check
     missing range check. (eli)
     missing range check. (eli)
 
 
-    HDFS-3541. Deadlock between recovery, xceiver and packet responder.
-    (Vinay via umamahesh)
-
     HDFS-3428. Move DelegationTokenRenewer to common (tucu)
     HDFS-3428. Move DelegationTokenRenewer to common (tucu)
 
 
     HDFS-3491. HttpFs does not set permissions correctly (tucu)
     HDFS-3491. HttpFs does not set permissions correctly (tucu)
@@ -1631,8 +1589,6 @@ Release 2.0.2-alpha - 2012-09-07
     HDFS-3609. libhdfs: don't force the URI to look like hdfs://hostname:port.
     HDFS-3609. libhdfs: don't force the URI to look like hdfs://hostname:port.
     (Colin Patrick McCabe via eli)
     (Colin Patrick McCabe via eli)
 
 
-    HDFS-2966 TestNameNodeMetrics tests can fail under load. (stevel)
-
     HDFS-3605. Block mistakenly marked corrupt during edit log catchup
     HDFS-3605. Block mistakenly marked corrupt during edit log catchup
     phase of failover. (todd and Brahma Reddy Battula via todd)
     phase of failover. (todd and Brahma Reddy Battula via todd)
 
 
@@ -1711,12 +1667,12 @@ Release 2.0.2-alpha - 2012-09-07
     HDFS-3715. Fix TestFileCreation#testFileCreationNamenodeRestart.
     HDFS-3715. Fix TestFileCreation#testFileCreationNamenodeRestart.
     (Andrew Whang via eli)
     (Andrew Whang via eli)
 
 
+    HDFS-3731. 2.0 release upgrade must handle blocks being written from 1.0.
+    (Colin Patrick McCabe via eli)
+
     HDFS-3683. Edit log replay progress indicator shows >100% complete. (Plamen
     HDFS-3683. Edit log replay progress indicator shows >100% complete. (Plamen
     Jeliazkov via atm)
     Jeliazkov via atm)
 
 
-    HDFS-3731. Release upgrade must handle blocks being written from 1.0.
-    (Colin Patrick McCabe via eli)
-
     HDFS-3856. TestHDFSServerPorts failure is causing surefire fork failure.
     HDFS-3856. TestHDFSServerPorts failure is causing surefire fork failure.
     (eli)
     (eli)
 
 
@@ -1734,16 +1690,9 @@ Release 2.0.2-alpha - 2012-09-07
     HDFS-3733. Audit logs should include WebHDFS access. (Andy Isaacson via 
     HDFS-3733. Audit logs should include WebHDFS access. (Andy Isaacson via 
     eli)
     eli)
 
 
-    HDFS-2686. Remove DistributedUpgrade related code. (suresh)
-
-    HDFS-3833. TestDFSShell fails on windows due to concurrent file 
-    read/write. (Brandon Li via suresh)
-
     HDFS-3466. Get HTTP kerberos principal from the web authentication keytab.
     HDFS-3466. Get HTTP kerberos principal from the web authentication keytab.
     (omalley)
     (omalley)
 
 
-    HDFS-3879. Fix findbugs warning in TransferFsImage on branch-2. (eli)
-
     HDFS-3469. start-dfs.sh will start zkfc, but stop-dfs.sh will not stop zkfc similarly.
     HDFS-3469. start-dfs.sh will start zkfc, but stop-dfs.sh will not stop zkfc similarly.
     (Vinay via umamahesh)
     (Vinay via umamahesh)
 
 
@@ -1787,6 +1736,9 @@ Release 2.0.2-alpha - 2012-09-07
     
     
     HDFS-3432. TestDFSZKFailoverController tries to fail over too early (todd)
     HDFS-3432. TestDFSZKFailoverController tries to fail over too early (todd)
 
 
+    HDFS-3833. TestDFSShell fails on windows due to concurrent file 
+    read/write. (Brandon Li via suresh)
+
     HDFS-3902. TestDatanodeBlockScanner#testBlockCorruptionPolicy is broken.
     HDFS-3902. TestDatanodeBlockScanner#testBlockCorruptionPolicy is broken.
     (Andy Isaacson via eli)
     (Andy Isaacson via eli)
 
 

+ 4 - 66
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -2282,68 +2282,10 @@ public class FSDirectory implements Closeable {
   }
   }
   
   
   /* This method is always called with writeLock held */
   /* This method is always called with writeLock held */
-  private final void removeFromInodeMap(INode inode) {
-    inodeMap.remove(inode);
-  }
-  
-  /** Remove all the inodes under given inode from the map */
-  private void remvoedAllFromInodesFromMap(INode inode) {
-    removeFromInodeMap(inode);
-    if (!inode.isDirectory()) {
-      return;
-    }
-    INodeDirectory dir = (INodeDirectory) inode;
-    for (INode child : dir.getChildrenList()) {
-      remvoedAllFromInodesFromMap(child);
-    }
-    dir.clearChildren();
-  }
-  
-  /** Update the count of each directory with quota in the namespace
-   * A directory's count is defined as the total number inodes in the tree
-   * rooted at the directory.
-   * 
-   * This is an update of existing state of the filesystem and does not
-   * throw QuotaExceededException.
-   */
-  void updateCountForINodeWithQuota() {
-    updateCountForINodeWithQuota(this, rootDir, new INode.DirCounts(), 
-                                 new ArrayList<INode>(50));
-  }
-  
-  /** 
-   * Update the count of the directory if it has a quota and return the count
-   * 
-   * This does not throw a QuotaExceededException. This is just an update
-   * of of existing state and throwing QuotaExceededException does not help
-   * with fixing the state, if there is a problem.
-   * 
-   * @param dir the root of the tree that represents the directory
-   * @param counters counters for name space and disk space
-   * @param nodesInPath INodes for the each of components in the path.
-   */
-  private static void updateCountForINodeWithQuota(FSDirectory fsd,
-      INodeDirectory dir, INode.DirCounts counts, ArrayList<INode> nodesInPath) {
-    long parentNamespace = counts.nsCount;
-    long parentDiskspace = counts.dsCount;
-    
-    counts.nsCount = 1L;//for self. should not call node.spaceConsumedInTree()
-    counts.dsCount = 0L;
-    
-    /* We don't need nodesInPath if we could use 'parent' field in 
-     * INode. using 'parent' is not currently recommended. */
-    nodesInPath.add(dir);
-
-    for (INode child : dir.getChildrenList()) {
-      fsd.inodeMap.put(child);
-      if (child.isDirectory()) {
-        updateCountForINodeWithQuota(fsd, (INodeDirectory)child, 
-                                     counts, nodesInPath);
-      } else if (child.isSymlink()) {
-        counts.nsCount += 1;
-      } else { // reduce recursive calls
-        counts.nsCount += 1;
-        counts.dsCount += ((INodeFile)child).diskspaceConsumed();
+  final void removeFromInodeMap(List<INode> inodes) {
+    if (inodes != null) {
+      for (INode inode : inodes) {
+        inodeMap.remove(inode);
       }
       }
     }
     }
   }
   }
@@ -2406,14 +2348,10 @@ public class FSDirectory implements Closeable {
         } else if (!quotaNode.isQuotaSet() && latest == null) {
         } else if (!quotaNode.isQuotaSet() && latest == null) {
           // will not come here for root because root's nsQuota is always set
           // will not come here for root because root's nsQuota is always set
           return quotaNode.replaceSelf4INodeDirectory();
           return quotaNode.replaceSelf4INodeDirectory();
-          // update the inodeMap
-          inodeMap.put(newNode);
         }
         }
       } else {
       } else {
         // a non-quota directory; so replace it with a directory with quota
         // a non-quota directory; so replace it with a directory with quota
         return dirNode.replaceSelf4Quota(latest, nsQuota, dsQuota);
         return dirNode.replaceSelf4Quota(latest, nsQuota, dsQuota);
-        // update the inodeMap
-        inodeMap.put(newNode);
       }
       }
       return (oldNsQuota != nsQuota || oldDsQuota != dsQuota) ? dirNode : null;
       return (oldNsQuota != nsQuota || oldDsQuota != dsQuota) ? dirNode : null;
     }
     }

+ 1 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java

@@ -118,17 +118,11 @@ public class NNStorageRetentionManager {
             .result();
             .result();
       }
       }
     });
     });
-
-    // Remove from consideration any edit logs that are in fact required.
-    while (editLogs.size() > 0 &&
-        editLogs.get(editLogs.size() - 1).getFirstTxId() >= minimumRequiredTxId) {
-      editLogs.remove(editLogs.size() - 1);
-    }
     
     
     // Next, adjust the number of transactions to retain if doing so would mean
     // Next, adjust the number of transactions to retain if doing so would mean
     // keeping too many segments around.
     // keeping too many segments around.
     while (editLogs.size() > maxExtraEditsSegmentsToRetain) {
     while (editLogs.size() > maxExtraEditsSegmentsToRetain) {
-      purgeLogsFrom = editLogs.get(0).getLastTxId() + 1;
+      purgeLogsFrom = editLogs.get(0).getFirstTxId();
       editLogs.remove(0);
       editLogs.remove(0);
     }
     }
     
     

+ 4 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java

@@ -146,15 +146,7 @@ public class TestDataTransferKeepalive {
       stm.read();
       stm.read();
       assertXceiverCount(1);
       assertXceiverCount(1);
 
 
-      // Poll for 0 running xceivers.  Allow up to 5 seconds for some slack.
-      long totalSleepTime = 0;
-      long sleepTime = WRITE_TIMEOUT + 100;
-      while (getXceiverCountWithoutServer() > 0 && totalSleepTime < 5000) {
-        Thread.sleep(sleepTime);
-        totalSleepTime += sleepTime;
-        sleepTime = 100;
-      }
-
+      Thread.sleep(WRITE_TIMEOUT + 1000);
       // DN should time out in sendChunks, and this should force
       // DN should time out in sendChunks, and this should force
       // the xceiver to exit.
       // the xceiver to exit.
       assertXceiverCount(0);
       assertXceiverCount(0);
@@ -198,7 +190,9 @@ public class TestDataTransferKeepalive {
   }
   }
 
 
   private void assertXceiverCount(int expected) {
   private void assertXceiverCount(int expected) {
-    int count = getXceiverCountWithoutServer();
+    // Subtract 1, since the DataXceiverServer
+    // counts as one
+    int count = dn.getXceiverCount() - 1;
     if (count != expected) {
     if (count != expected) {
       ReflectionUtils.printThreadInfo(
       ReflectionUtils.printThreadInfo(
           new PrintWriter(System.err),
           new PrintWriter(System.err),
@@ -207,14 +201,4 @@ public class TestDataTransferKeepalive {
           count);
           count);
     }
     }
   }
   }
-
-  /**
-   * Returns the datanode's xceiver count, but subtracts 1, since the
-   * DataXceiverServer counts as one.
-   * 
-   * @return int xceiver count, not including DataXceiverServer
-   */
-  private int getXceiverCountWithoutServer() {
-    return dn.getXceiverCount() - 1;
-  }
 }
 }

+ 1 - 37
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -19,9 +19,8 @@
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
 
 
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
@@ -863,39 +862,4 @@ public class TestINodeFile {
     resolvedPath = FSDirectory.resolvePath(testPath, components, fsd);
     resolvedPath = FSDirectory.resolvePath(testPath, components, fsd);
     assertEquals(testPath, resolvedPath);
     assertEquals(testPath, resolvedPath);
   }
   }
-  
-  /**
-   * Test whether the inode in inodeMap has been replaced after regular inode
-   * replacement
-   */
-  @Test
-  public void testInodeReplacement() throws Exception {
-    final Configuration conf = new Configuration();
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).
-        numDataNodes(1).build();
-    cluster.waitActive();
-    final DistributedFileSystem hdfs = cluster.getFileSystem();
-    final FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
-    
-    final Path dir = new Path("/dir");
-    hdfs.mkdirs(dir);
-    INode dirNode = fsdir.getINode(dir.toString());
-    INode dirNodeFromNode = fsdir.getInode(dirNode.getId());
-    assertSame(dirNode, dirNodeFromNode);
-    
-    // set quota to dir, which leads to node replacement
-    hdfs.setQuota(dir, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
-    dirNode = fsdir.getINode(dir.toString());
-    assertTrue(dirNode instanceof INodeDirectoryWithQuota);
-    // the inode in inodeMap should also be replaced
-    dirNodeFromNode = fsdir.getInode(dirNode.getId());
-    assertSame(dirNode, dirNodeFromNode);
-    
-    hdfs.setQuota(dir, -1, -1);
-    dirNode = fsdir.getINode(dir.toString());
-    assertTrue(dirNode instanceof INodeDirectory);
-    // the inode in inodeMap should also be replaced
-    dirNodeFromNode = fsdir.getInode(dirNode.getId());
-    assertSame(dirNode, dirNodeFromNode);
-  }
 }
 }

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java

@@ -217,12 +217,11 @@ public class TestNNStorageRetentionManager {
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(176, 200), true);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(176, 200), true);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(201, 225), true);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(201, 225), true);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(226, 240), true);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(226, 240), true);
-    // Only retain 2 extra segments. The 301-350 and 351-400 segments are
-    // considered required, not extra.
+    // Only retain 2 extra segments. The 301-400 segment is considered required,
+    // not extra.
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(241, 275), false);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(241, 275), false);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(276, 300), false);
     tc.addLog("/foo2/current/" + getFinalizedEditsFileName(276, 300), false);
-    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(301, 350), false);
-    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(351, 400), false);
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(301, 400), false);
     tc.addLog("/foo2/current/" + getInProgressEditsFileName(401), false);
     tc.addLog("/foo2/current/" + getInProgressEditsFileName(401), false);
     runTest(tc);
     runTest(tc);
   }
   }

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

@@ -209,9 +209,6 @@ Release 2.0.5-beta - UNRELEASED
     MAPREDUCE-5175. Updated MR App to not set envs that will be set by NMs
     MAPREDUCE-5175. Updated MR App to not set envs that will be set by NMs
     anyways after YARN-561. (Xuan Gong via vinodkv)
     anyways after YARN-561. (Xuan Gong via vinodkv)
 
 
-    MAPREDUCE-5069. add concrete common implementations of
-    CombineFileInputFormat (Sangjin Lee via bobby)
-
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     MAPREDUCE-4974. Optimising the LineRecordReader initialize() method 
     MAPREDUCE-4974. Optimising the LineRecordReader initialize() method 
@@ -339,23 +336,6 @@ Release 2.0.5-beta - UNRELEASED
     MAPREDUCE-5146. application classloader may be used too early to load
     MAPREDUCE-5146. application classloader may be used too early to load
     classes. (Sangjin Lee via tomwhite)
     classes. (Sangjin Lee via tomwhite)
 
 
-    MAPREDUCE-4737. Ensure that mapreduce APIs are semantically consistent
-    with mapred API w.r.t Mapper.cleanup and Reducer.cleanup; in the sense that
-    cleanup is now called even if there is an error. The old mapred API
-    already ensures that Mapper.close and Reducer.close are invoked during
-    error handling. Note that it is an incompatible change, however end-users 
-    can override Mapper.run and Reducer.run to get the old (inconsistent) 
-    behaviour. (acmurthy)
-
-    MAPREDUCE-5166. Fix ConcurrentModificationException due to insufficient
-    synchronization on updates to task Counters. (Sandy Ryza via acmurthy)
-
-    MAPREDUCE-5181. RMCommunicator should not use AMToken from the env.
-    (Vinod Kumar Vavilapalli via sseth)
-
-    MAPREDUCE-5178. Update MR App to set progress in ApplicationReport after
-    YARN-577. (Hitesh Shah via vinodkv)
-
 Release 2.0.4-alpha - UNRELEASED
 Release 2.0.4-alpha - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 22 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -38,9 +38,13 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
@@ -276,7 +280,24 @@ public abstract class RMCommunicator extends AbstractService
       throw new YarnException(e);
       throw new YarnException(e);
     }
     }
 
 
-    // CurrentUser should already have AMToken loaded.
+    if (UserGroupInformation.isSecurityEnabled()) {
+      String tokenURLEncodedStr = System.getenv().get(
+          ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME);
+      Token<? extends TokenIdentifier> token = new Token<TokenIdentifier>();
+
+      try {
+        token.decodeFromUrlString(tokenURLEncodedStr);
+      } catch (IOException e) {
+        throw new YarnException(e);
+      }
+
+      SecurityUtil.setTokenService(token, serviceAddr);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("AppMasterToken is " + token);
+      }
+      currentUser.addToken(token);
+    }
+
     return currentUser.doAs(new PrivilegedAction<AMRMProtocol>() {
     return currentUser.doAs(new PrivilegedAction<AMRMProtocol>() {
       @Override
       @Override
       public AMRMProtocol run() {
       public AMRMProtocol run() {

+ 6 - 17
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -18,10 +18,6 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -90,6 +86,8 @@ public class LocalJobRunner implements ClientProtocol {
 
 
   private static final String jobDir =  "localRunner/";
   private static final String jobDir =  "localRunner/";
 
 
+  private static final Counters EMPTY_COUNTERS = new Counters();
+
   public long getProtocolVersion(String protocol, long clientVersion) {
   public long getProtocolVersion(String protocol, long clientVersion) {
     return ClientProtocol.versionID;
     return ClientProtocol.versionID;
   }
   }
@@ -275,10 +273,10 @@ public class LocalJobRunner implements ClientProtocol {
       this.partialMapProgress = new float[numMaps];
       this.partialMapProgress = new float[numMaps];
       this.mapCounters = new Counters[numMaps];
       this.mapCounters = new Counters[numMaps];
       for (int i = 0; i < numMaps; i++) {
       for (int i = 0; i < numMaps; i++) {
-        this.mapCounters[i] = new Counters();
+        this.mapCounters[i] = EMPTY_COUNTERS;
       }
       }
 
 
-      this.reduceCounters = new Counters();
+      this.reduceCounters = EMPTY_COUNTERS;
     }
     }
 
 
     /**
     /**
@@ -499,15 +497,6 @@ public class LocalJobRunner implements ClientProtocol {
     
     
     public synchronized boolean statusUpdate(TaskAttemptID taskId,
     public synchronized boolean statusUpdate(TaskAttemptID taskId,
         TaskStatus taskStatus) throws IOException, InterruptedException {
         TaskStatus taskStatus) throws IOException, InterruptedException {
-      // Serialize as we would if distributed in order to make deep copy
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
-      DataOutputStream dos = new DataOutputStream(baos);
-      taskStatus.write(dos);
-      dos.close();
-      taskStatus = TaskStatus.createTaskStatus(taskStatus.getIsMap());
-      taskStatus.readFields(new DataInputStream(
-          new ByteArrayInputStream(baos.toByteArray())));
-      
       LOG.info(taskStatus.getStateString());
       LOG.info(taskStatus.getStateString());
       int taskIndex = mapIds.indexOf(taskId);
       int taskIndex = mapIds.indexOf(taskId);
       if (taskIndex >= 0) {                       // mapping
       if (taskIndex >= 0) {                       // mapping
@@ -536,10 +525,10 @@ public class LocalJobRunner implements ClientProtocol {
     public synchronized Counters getCurrentCounters() {
     public synchronized Counters getCurrentCounters() {
       if (null == mapCounters) {
       if (null == mapCounters) {
         // Counters not yet initialized for job.
         // Counters not yet initialized for job.
-        return new Counters();
+        return EMPTY_COUNTERS;
       }
       }
 
 
-      Counters current = new Counters();
+      Counters current = EMPTY_COUNTERS;
       for (Counters c : mapCounters) {
       for (Counters c : mapCounters) {
         current = Counters.sum(current, c);
         current = Counters.sum(current, c);
       }
       }

+ 10 - 73
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java

@@ -434,15 +434,10 @@ public class MapTask extends Task {
       }
       }
       statusUpdate(umbilical);
       statusUpdate(umbilical);
       collector.flush();
       collector.flush();
-      
-      in.close();
-      in = null;
-      
-      collector.close();
-      collector = null;
     } finally {
     } finally {
-      closeQuietly(in);
-      closeQuietly(collector);
+      //close
+      in.close();                               // close input
+      collector.close();
     }
     }
   }
   }
 
 
@@ -758,20 +753,13 @@ public class MapTask extends Task {
           new WrappedMapper<INKEY, INVALUE, OUTKEY, OUTVALUE>().getMapContext(
           new WrappedMapper<INKEY, INVALUE, OUTKEY, OUTVALUE>().getMapContext(
               mapContext);
               mapContext);
 
 
-    try {
-      input.initialize(split, mapperContext);
-      mapper.run(mapperContext);
-      mapPhase.complete();
-      setPhase(TaskStatus.Phase.SORT);
-      statusUpdate(umbilical);
-      input.close();
-      input = null;
-      output.close(mapperContext);
-      output = null;
-    } finally {
-      closeQuietly(input);
-      closeQuietly(output, mapperContext);
-    }
+    input.initialize(split, mapperContext);
+    mapper.run(mapperContext);
+    mapPhase.complete();
+    setPhase(TaskStatus.Phase.SORT);
+    statusUpdate(umbilical);
+    input.close();
+    output.close(mapperContext);
   }
   }
 
 
   class DirectMapOutputCollector<K, V>
   class DirectMapOutputCollector<K, V>
@@ -1961,55 +1949,4 @@ public class MapTask extends Task {
     }
     }
   }
   }
 
 
-  private <INKEY,INVALUE,OUTKEY,OUTVALUE>
-  void closeQuietly(RecordReader<INKEY, INVALUE> c) {
-    if (c != null) {
-      try {
-        c.close();
-      } catch (IOException ie) {
-        // Ignore
-        LOG.info("Ignoring exception during close for " + c, ie);
-      }
-    }
-  }
-  
-  private <OUTKEY, OUTVALUE>
-  void closeQuietly(MapOutputCollector<OUTKEY, OUTVALUE> c) {
-    if (c != null) {
-      try {
-        c.close();
-      } catch (Exception ie) {
-        // Ignore
-        LOG.info("Ignoring exception during close for " + c, ie);
-      }
-    }
-  }
-  
-  private <INKEY, INVALUE, OUTKEY, OUTVALUE>
-  void closeQuietly(
-      org.apache.hadoop.mapreduce.RecordReader<INKEY, INVALUE> c) {
-    if (c != null) {
-      try {
-        c.close();
-      } catch (Exception ie) {
-        // Ignore
-        LOG.info("Ignoring exception during close for " + c, ie);
-      }
-    }
-  }
-
-  private <INKEY, INVALUE, OUTKEY, OUTVALUE>
-  void closeQuietly(
-      org.apache.hadoop.mapreduce.RecordWriter<OUTKEY, OUTVALUE> c,
-      org.apache.hadoop.mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context
-          mapperContext) {
-    if (c != null) {
-      try {
-        c.close(mapperContext);
-      } catch (Exception ie) {
-        // Ignore
-        LOG.info("Ignoring exception during close for " + c, ie);
-      }
-    }
-  }
 }
 }

+ 17 - 27
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
@@ -429,15 +428,14 @@ public class ReduceTask extends Task {
     // make output collector
     // make output collector
     String finalName = getOutputName(getPartition());
     String finalName = getOutputName(getPartition());
 
 
-    RecordWriter<OUTKEY, OUTVALUE> out = new OldTrackingRecordWriter<OUTKEY, OUTVALUE>(
+    final RecordWriter<OUTKEY, OUTVALUE> out = new OldTrackingRecordWriter<OUTKEY, OUTVALUE>(
         this, job, reporter, finalName);
         this, job, reporter, finalName);
-    final RecordWriter<OUTKEY, OUTVALUE> finalOut = out;
-    
+
     OutputCollector<OUTKEY,OUTVALUE> collector = 
     OutputCollector<OUTKEY,OUTVALUE> collector = 
       new OutputCollector<OUTKEY,OUTVALUE>() {
       new OutputCollector<OUTKEY,OUTVALUE>() {
         public void collect(OUTKEY key, OUTVALUE value)
         public void collect(OUTKEY key, OUTVALUE value)
           throws IOException {
           throws IOException {
-          finalOut.write(key, value);
+          out.write(key, value);
           // indicate that progress update needs to be sent
           // indicate that progress update needs to be sent
           reporter.progress();
           reporter.progress();
         }
         }
@@ -468,14 +466,20 @@ public class ReduceTask extends Task {
         values.informReduceProgress();
         values.informReduceProgress();
       }
       }
 
 
+      //Clean up: repeated in catch block below
       reducer.close();
       reducer.close();
-      reducer = null;
-      
       out.close(reporter);
       out.close(reporter);
-      out = null;
-    } finally {
-      IOUtils.cleanup(LOG, reducer);
-      closeQuietly(out, reporter);
+      //End of clean up.
+    } catch (IOException ioe) {
+      try {
+        reducer.close();
+      } catch (IOException ignored) {}
+        
+      try {
+        out.close(reporter);
+      } catch (IOException ignored) {}
+      
+      throw ioe;
     }
     }
   }
   }
 
 
@@ -641,21 +645,7 @@ public class ReduceTask extends Task {
                                                committer,
                                                committer,
                                                reporter, comparator, keyClass,
                                                reporter, comparator, keyClass,
                                                valueClass);
                                                valueClass);
-    try {
-      reducer.run(reducerContext);
-    } finally {
-      trackedRW.close(reducerContext);
-    }
-  }
-  
-  private <OUTKEY, OUTVALUE>
-  void closeQuietly(RecordWriter<OUTKEY, OUTVALUE> c, Reporter r) {
-    if (c != null) {
-      try {
-        c.close(r);
-      } catch (Exception e) {
-        LOG.info("Exception in closing " + c, e);
-      }
-    }
+    reducer.run(reducerContext);
+    trackedRW.close(reducerContext);
   }
   }
 }
 }

+ 0 - 86
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReaderWrapper.java

@@ -1,86 +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.mapred.lib;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * A wrapper class for a record reader that handles a single file split. It
- * delegates most of the methods to the wrapped instance. A concrete subclass
- * needs to provide a constructor that calls this parent constructor with the
- * appropriate input format. The subclass constructor must satisfy the specific
- * constructor signature that is required by
- * <code>CombineFileRecordReader</code>.
- *
- * Subclassing is needed to get a concrete record reader wrapper because of the
- * constructor requirement.
- *
- * @see CombineFileRecordReader
- * @see CombineFileInputFormat
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public abstract class CombineFileRecordReaderWrapper<K,V>
-  implements RecordReader<K,V> {
-  private final RecordReader<K,V> delegate;
-
-  protected CombineFileRecordReaderWrapper(FileInputFormat<K,V> inputFormat,
-    CombineFileSplit split, Configuration conf, Reporter reporter, Integer idx)
-    throws IOException {
-    FileSplit fileSplit = new FileSplit(split.getPath(idx),
-      split.getOffset(idx),
-      split.getLength(idx),
-      split.getLocations());
-
-    delegate = inputFormat.getRecordReader(fileSplit, (JobConf)conf, reporter);
-  }
-
-  public boolean next(K key, V value) throws IOException {
-    return delegate.next(key, value);
-  }
-
-  public K createKey() {
-    return delegate.createKey();
-  }
-
-  public V createValue() {
-    return delegate.createValue();
-  }
-
-  public long getPos() throws IOException {
-    return delegate.getPos();
-  }
-
-  public void close() throws IOException {
-    delegate.close();
-  }
-
-  public float getProgress() throws IOException {
-    return delegate.getProgress();
-  }
-}

+ 0 - 66
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineSequenceFileInputFormat.java

@@ -1,66 +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.mapred.lib;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.SequenceFileInputFormat;
-
-/**
- * Input format that is a <code>CombineFileInputFormat</code>-equivalent for
- * <code>SequenceFileInputFormat</code>.
- *
- * @see CombineFileInputFormat
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class CombineSequenceFileInputFormat<K,V>
-  extends CombineFileInputFormat<K,V> {
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  public RecordReader<K,V> getRecordReader(InputSplit split, JobConf conf,
-    Reporter reporter) throws IOException {
-    return new CombineFileRecordReader(conf, (CombineFileSplit)split, reporter,
-      SequenceFileRecordReaderWrapper.class);
-  }
-
-  /**
-   * A record reader that may be passed to <code>CombineFileRecordReader</code>
-   * so that it can be used in a <code>CombineFileInputFormat</code>-equivalent
-   * for <code>SequenceFileInputFormat</code>.
-   *
-   * @see CombineFileRecordReader
-   * @see CombineFileInputFormat
-   * @see SequenceFileInputFormat
-   */
-  private static class SequenceFileRecordReaderWrapper<K,V>
-    extends CombineFileRecordReaderWrapper<K,V> {
-    // this constructor signature is required by CombineFileRecordReader
-    public SequenceFileRecordReaderWrapper(CombineFileSplit split,
-      Configuration conf, Reporter reporter, Integer idx) throws IOException {
-      super(new SequenceFileInputFormat<K,V>(), split, conf, reporter, idx);
-    }
-  }
-}

+ 0 - 68
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineTextInputFormat.java

@@ -1,68 +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.mapred.lib;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.TextInputFormat;
-
-/**
- * Input format that is a <code>CombineFileInputFormat</code>-equivalent for
- * <code>TextInputFormat</code>.
- *
- * @see CombineFileInputFormat
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class CombineTextInputFormat
-  extends CombineFileInputFormat<LongWritable,Text> {
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  public RecordReader<LongWritable,Text> getRecordReader(InputSplit split,
-    JobConf conf, Reporter reporter) throws IOException {
-    return new CombineFileRecordReader(conf, (CombineFileSplit)split, reporter,
-      TextRecordReaderWrapper.class);
-  }
-
-  /**
-   * A record reader that may be passed to <code>CombineFileRecordReader</code>
-   * so that it can be used in a <code>CombineFileInputFormat</code>-equivalent
-   * for <code>TextInputFormat</code>.
-   *
-   * @see CombineFileRecordReader
-   * @see CombineFileInputFormat
-   * @see TextInputFormat
-   */
-  private static class TextRecordReaderWrapper
-    extends CombineFileRecordReaderWrapper<LongWritable,Text> {
-    // this constructor signature is required by CombineFileRecordReader
-    public TextRecordReaderWrapper(CombineFileSplit split, Configuration conf,
-      Reporter reporter, Integer idx) throws IOException {
-      super(new TextInputFormat(), split, conf, reporter, idx);
-    }
-  }
-}

+ 3 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java

@@ -140,12 +140,9 @@ public class Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
    */
    */
   public void run(Context context) throws IOException, InterruptedException {
   public void run(Context context) throws IOException, InterruptedException {
     setup(context);
     setup(context);
-    try {
-      while (context.nextKeyValue()) {
-        map(context.getCurrentKey(), context.getCurrentValue(), context);
-      }
-    } finally {
-      cleanup(context);
+    while (context.nextKeyValue()) {
+      map(context.getCurrentKey(), context.getCurrentValue(), context);
     }
     }
+    cleanup(context);
   }
   }
 }
 }

+ 7 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java

@@ -166,17 +166,14 @@ public class Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
    */
    */
   public void run(Context context) throws IOException, InterruptedException {
   public void run(Context context) throws IOException, InterruptedException {
     setup(context);
     setup(context);
-    try {
-      while (context.nextKey()) {
-        reduce(context.getCurrentKey(), context.getValues(), context);
-        // If a back up store is used, reset it
-        Iterator<VALUEIN> iter = context.getValues().iterator();
-        if(iter instanceof ReduceContext.ValueIterator) {
-          ((ReduceContext.ValueIterator<VALUEIN>)iter).resetBackupStore();        
-        }
+    while (context.nextKey()) {
+      reduce(context.getCurrentKey(), context.getValues(), context);
+      // If a back up store is used, reset it
+      Iterator<VALUEIN> iter = context.getValues().iterator();
+      if(iter instanceof ReduceContext.ValueIterator) {
+        ((ReduceContext.ValueIterator<VALUEIN>)iter).resetBackupStore();        
       }
       }
-    } finally {
-      cleanup(context);
     }
     }
+    cleanup(context);
   }
   }
 }
 }

+ 0 - 108
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileRecordReaderWrapper.java

@@ -1,108 +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.mapreduce.lib.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * A wrapper class for a record reader that handles a single file split. It
- * delegates most of the methods to the wrapped instance. A concrete subclass
- * needs to provide a constructor that calls this parent constructor with the
- * appropriate input format. The subclass constructor must satisfy the specific
- * constructor signature that is required by
- * <code>CombineFileRecordReader</code>.
- *
- * Subclassing is needed to get a concrete record reader wrapper because of the
- * constructor requirement.
- *
- * @see CombineFileRecordReader
- * @see CombineFileInputFormat
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public abstract class CombineFileRecordReaderWrapper<K,V>
-  extends RecordReader<K,V> {
-  private final FileSplit fileSplit;
-  private final RecordReader<K,V> delegate;
-
-  protected CombineFileRecordReaderWrapper(FileInputFormat<K,V> inputFormat,
-    CombineFileSplit split, TaskAttemptContext context, Integer idx)
-    throws IOException, InterruptedException {
-    fileSplit = new FileSplit(split.getPath(idx),
-      split.getOffset(idx),
-      split.getLength(idx),
-      split.getLocations());
-
-    delegate = inputFormat.createRecordReader(fileSplit, context);
-  }
-
-  public void initialize(InputSplit split, TaskAttemptContext context)
-    throws IOException, InterruptedException {
-    // it really should be the same file split at the time the wrapper instance
-    // was created
-    assert fileSplitIsValid(context);
-
-    delegate.initialize(fileSplit, context);
-  }
-
-  private boolean fileSplitIsValid(TaskAttemptContext context) {
-    Configuration conf = context.getConfiguration();
-    long offset = conf.getLong(MRJobConfig.MAP_INPUT_START, 0L);
-    if (fileSplit.getStart() != offset) {
-      return false;
-    }
-    long length = conf.getLong(MRJobConfig.MAP_INPUT_PATH, 0L);
-    if (fileSplit.getLength() != length) {
-      return false;
-    }
-    String path = conf.get(MRJobConfig.MAP_INPUT_FILE);
-    if (!fileSplit.getPath().toString().equals(path)) {
-      return false;
-    }
-    return true;
-  }
-
-  public boolean nextKeyValue() throws IOException, InterruptedException {
-    return delegate.nextKeyValue();
-  }
-
-  public K getCurrentKey() throws IOException, InterruptedException {
-    return delegate.getCurrentKey();
-  }
-
-  public V getCurrentValue() throws IOException, InterruptedException {
-    return delegate.getCurrentValue();
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    return delegate.getProgress();
-  }
-
-  public void close() throws IOException {
-    delegate.close();
-  }
-}

+ 0 - 64
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineSequenceFileInputFormat.java

@@ -1,64 +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.mapreduce.lib.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * Input format that is a <code>CombineFileInputFormat</code>-equivalent for
- * <code>SequenceFileInputFormat</code>.
- *
- * @see CombineFileInputFormat
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class CombineSequenceFileInputFormat<K,V>
-  extends CombineFileInputFormat<K,V> {
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  public RecordReader<K,V> createRecordReader(InputSplit split,
-    TaskAttemptContext context) throws IOException {
-    return new CombineFileRecordReader((CombineFileSplit)split, context,
-      SequenceFileRecordReaderWrapper.class);
-  }
-
-  /**
-   * A record reader that may be passed to <code>CombineFileRecordReader</code>
-   * so that it can be used in a <code>CombineFileInputFormat</code>-equivalent
-   * for <code>SequenceFileInputFormat</code>.
-   *
-   * @see CombineFileRecordReader
-   * @see CombineFileInputFormat
-   * @see SequenceFileInputFormat
-   */
-  private static class SequenceFileRecordReaderWrapper<K,V>
-    extends CombineFileRecordReaderWrapper<K,V> {
-    // this constructor signature is required by CombineFileRecordReader
-    public SequenceFileRecordReaderWrapper(CombineFileSplit split,
-      TaskAttemptContext context, Integer idx)
-      throws IOException, InterruptedException {
-      super(new SequenceFileInputFormat<K,V>(), split, context, idx);
-    }
-  }
-}

+ 0 - 65
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineTextInputFormat.java

@@ -1,65 +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.mapreduce.lib.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * Input format that is a <code>CombineFileInputFormat</code>-equivalent for
- * <code>TextInputFormat</code>.
- *
- * @see CombineFileInputFormat
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class CombineTextInputFormat
-  extends CombineFileInputFormat<LongWritable,Text> {
-  public RecordReader<LongWritable,Text> createRecordReader(InputSplit split,
-    TaskAttemptContext context) throws IOException {
-    return new CombineFileRecordReader<LongWritable,Text>(
-      (CombineFileSplit)split, context, TextRecordReaderWrapper.class);
-  }
-
-  /**
-   * A record reader that may be passed to <code>CombineFileRecordReader</code>
-   * so that it can be used in a <code>CombineFileInputFormat</code>-equivalent
-   * for <code>TextInputFormat</code>.
-   *
-   * @see CombineFileRecordReader
-   * @see CombineFileInputFormat
-   * @see TextInputFormat
-   */
-  private static class TextRecordReaderWrapper
-    extends CombineFileRecordReaderWrapper<LongWritable,Text> {
-    // this constructor signature is required by CombineFileRecordReader
-    public TextRecordReaderWrapper(CombineFileSplit split,
-      TaskAttemptContext context, Integer idx)
-      throws IOException, InterruptedException {
-      super(new TextInputFormat(), split, context, idx);
-    }
-  }
-}

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

@@ -89,7 +89,7 @@ public class NotRunningJob implements MRClientProtocol {
     // used for a non running job
     // used for a non running job
     return BuilderUtils.newApplicationReport(unknownAppId, unknownAttemptId,
     return BuilderUtils.newApplicationReport(unknownAppId, unknownAttemptId,
         "N/A", "N/A", "N/A", "N/A", 0, null, YarnApplicationState.NEW, "N/A",
         "N/A", "N/A", "N/A", "N/A", 0, null, YarnApplicationState.NEW, "N/A",
-        "N/A", 0, 0, FinalApplicationStatus.UNDEFINED, null, "N/A", 0.0f);
+        "N/A", 0, 0, FinalApplicationStatus.UNDEFINED, null, "N/A");
   }
   }
 
 
   NotRunningJob(ApplicationReport applicationReport, JobState jobState) {
   NotRunningJob(ApplicationReport applicationReport, JobState jobState) {

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java

@@ -413,7 +413,7 @@ public class TestClientServiceDelegate {
     return BuilderUtils.newApplicationReport(appId, attemptId, "user", "queue",
     return BuilderUtils.newApplicationReport(appId, attemptId, "user", "queue",
         "appname", "host", 124, null, YarnApplicationState.FINISHED,
         "appname", "host", 124, null, YarnApplicationState.FINISHED,
         "diagnostics", "url", 0, 0, FinalApplicationStatus.SUCCEEDED, null,
         "diagnostics", "url", 0, 0, FinalApplicationStatus.SUCCEEDED, null,
-        "N/A", 0.0f);
+        "N/A");
   }
   }
 
 
   private ApplicationReport getRunningApplicationReport(String host, int port) {
   private ApplicationReport getRunningApplicationReport(String host, int port) {
@@ -423,7 +423,7 @@ public class TestClientServiceDelegate {
     return BuilderUtils.newApplicationReport(appId, attemptId, "user", "queue",
     return BuilderUtils.newApplicationReport(appId, attemptId, "user", "queue",
         "appname", host, port, null, YarnApplicationState.RUNNING,
         "appname", host, port, null, YarnApplicationState.RUNNING,
         "diagnostics", "url", 0, 0, FinalApplicationStatus.UNDEFINED, null,
         "diagnostics", "url", 0, 0, FinalApplicationStatus.UNDEFINED, null,
-        "N/A", 0.0f);
+        "N/A");
   }
   }
 
 
   private ResourceMgrDelegate getRMDelegate() throws YarnRemoteException {
   private ResourceMgrDelegate getRMDelegate() throws YarnRemoteException {

+ 0 - 170
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java

@@ -1,170 +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.mapred;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertFalse;
-
-import java.io.IOException;
-import java.util.BitSet;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.mapred.lib.CombineFileSplit;
-import org.apache.hadoop.mapred.lib.CombineSequenceFileInputFormat;
-import org.junit.Test;
-
-public class TestCombineSequenceFileInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestCombineSequenceFileInputFormat.class);
-
-  private static Configuration conf = new Configuration();
-  private static FileSystem localFs = null;
-
-  static {
-    try {
-      conf.set("fs.defaultFS", "file:///");
-      localFs = FileSystem.getLocal(conf);
-    } catch (IOException e) {
-      throw new RuntimeException("init failure", e);
-    }
-  }
-
-  @SuppressWarnings("deprecation")
-  private static Path workDir =
-    new Path(new Path(System.getProperty("test.build.data", "/tmp")),
-             "TestCombineSequenceFileInputFormat").makeQualified(localFs);
-
-  @Test(timeout=10000)
-  public void testFormat() throws Exception {
-    JobConf job = new JobConf(conf);
-
-    Reporter reporter = Reporter.NULL;
-
-    Random random = new Random();
-    long seed = random.nextLong();
-    LOG.info("seed = "+seed);
-    random.setSeed(seed);
-
-    localFs.delete(workDir, true);
-
-    FileInputFormat.setInputPaths(job, workDir);
-
-    final int length = 10000;
-    final int numFiles = 10;
-
-    // create a file with various lengths
-    createFiles(length, numFiles, random);
-
-    // create a combine split for the files
-    InputFormat<IntWritable, BytesWritable> format =
-      new CombineSequenceFileInputFormat<IntWritable, BytesWritable>();
-    IntWritable key = new IntWritable();
-    BytesWritable value = new BytesWritable();
-    for (int i = 0; i < 3; i++) {
-      int numSplits =
-        random.nextInt(length/(SequenceFile.SYNC_INTERVAL/20))+1;
-      LOG.info("splitting: requesting = " + numSplits);
-      InputSplit[] splits = format.getSplits(job, numSplits);
-      LOG.info("splitting: got =        " + splits.length);
-
-      // we should have a single split as the length is comfortably smaller than
-      // the block size
-      assertEquals("We got more than one splits!", 1, splits.length);
-      InputSplit split = splits[0];
-      assertEquals("It should be CombineFileSplit",
-        CombineFileSplit.class, split.getClass());
-
-      // check each split
-      BitSet bits = new BitSet(length);
-      RecordReader<IntWritable, BytesWritable> reader =
-        format.getRecordReader(split, job, reporter);
-      try {
-        while (reader.next(key, value)) {
-          assertFalse("Key in multiple partitions.", bits.get(key.get()));
-          bits.set(key.get());
-        }
-      } finally {
-        reader.close();
-      }
-      assertEquals("Some keys in no partition.", length, bits.cardinality());
-    }
-  }
-
-  private static class Range {
-    private final int start;
-    private final int end;
-
-    Range(int start, int end) {
-      this.start = start;
-      this.end = end;
-    }
-
-    @Override
-    public String toString() {
-      return "(" + start + ", " + end + ")";
-    }
-  }
-
-  private static Range[] createRanges(int length, int numFiles, Random random) {
-    // generate a number of files with various lengths
-    Range[] ranges = new Range[numFiles];
-    for (int i = 0; i < numFiles; i++) {
-      int start = i == 0 ? 0 : ranges[i-1].end;
-      int end = i == numFiles - 1 ?
-        length :
-        (length/numFiles)*(2*i + 1)/2 + random.nextInt(length/numFiles) + 1;
-      ranges[i] = new Range(start, end);
-    }
-    return ranges;
-  }
-
-  private static void createFiles(int length, int numFiles, Random random)
-    throws IOException {
-    Range[] ranges = createRanges(length, numFiles, random);
-
-    for (int i = 0; i < numFiles; i++) {
-      Path file = new Path(workDir, "test_" + i + ".seq");
-      // create a file with length entries
-      @SuppressWarnings("deprecation")
-      SequenceFile.Writer writer =
-        SequenceFile.createWriter(localFs, conf, file,
-                                  IntWritable.class, BytesWritable.class);
-      Range range = ranges[i];
-      try {
-        for (int j = range.start; j < range.end; j++) {
-          IntWritable key = new IntWritable(j);
-          byte[] data = new byte[random.nextInt(10)];
-          random.nextBytes(data);
-          BytesWritable value = new BytesWritable(data);
-          writer.append(key, value);
-        }
-      } finally {
-        writer.close();
-      }
-    }
-  }
-}

+ 0 - 250
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java

@@ -1,250 +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.mapred;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertFalse;
-import static junit.framework.Assert.fail;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.mapred.lib.CombineFileSplit;
-import org.apache.hadoop.mapred.lib.CombineTextInputFormat;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.junit.Test;
-
-public class TestCombineTextInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestCombineTextInputFormat.class);
-
-  private static JobConf defaultConf = new JobConf();
-  private static FileSystem localFs = null;
-
-  static {
-    try {
-      defaultConf.set("fs.defaultFS", "file:///");
-      localFs = FileSystem.getLocal(defaultConf);
-    } catch (IOException e) {
-      throw new RuntimeException("init failure", e);
-    }
-  }
-
-  @SuppressWarnings("deprecation")
-  private static Path workDir =
-    new Path(new Path(System.getProperty("test.build.data", "/tmp")),
-             "TestCombineTextInputFormat").makeQualified(localFs);
-
-  // A reporter that does nothing
-  private static final Reporter voidReporter = Reporter.NULL;
-
-  @Test(timeout=10000)
-  public void testFormat() throws Exception {
-    JobConf job = new JobConf(defaultConf);
-
-    Random random = new Random();
-    long seed = random.nextLong();
-    LOG.info("seed = "+seed);
-    random.setSeed(seed);
-
-    localFs.delete(workDir, true);
-    FileInputFormat.setInputPaths(job, workDir);
-
-    final int length = 10000;
-    final int numFiles = 10;
-
-    createFiles(length, numFiles, random);
-
-    // create a combined split for the files
-    CombineTextInputFormat format = new CombineTextInputFormat();
-    LongWritable key = new LongWritable();
-    Text value = new Text();
-    for (int i = 0; i < 3; i++) {
-      int numSplits = random.nextInt(length/20)+1;
-      LOG.info("splitting: requesting = " + numSplits);
-      InputSplit[] splits = format.getSplits(job, numSplits);
-      LOG.info("splitting: got =        " + splits.length);
-
-      // we should have a single split as the length is comfortably smaller than
-      // the block size
-      assertEquals("We got more than one splits!", 1, splits.length);
-      InputSplit split = splits[0];
-      assertEquals("It should be CombineFileSplit",
-        CombineFileSplit.class, split.getClass());
-
-      // check the split
-      BitSet bits = new BitSet(length);
-      LOG.debug("split= " + split);
-      RecordReader<LongWritable, Text> reader =
-        format.getRecordReader(split, job, voidReporter);
-      try {
-        int count = 0;
-        while (reader.next(key, value)) {
-          int v = Integer.parseInt(value.toString());
-          LOG.debug("read " + v);
-          if (bits.get(v)) {
-            LOG.warn("conflict with " + v +
-                     " at position "+reader.getPos());
-          }
-          assertFalse("Key in multiple partitions.", bits.get(v));
-          bits.set(v);
-          count++;
-        }
-        LOG.info("splits="+split+" count=" + count);
-      } finally {
-        reader.close();
-      }
-      assertEquals("Some keys in no partition.", length, bits.cardinality());
-    }
-  }
-
-  private static class Range {
-    private final int start;
-    private final int end;
-
-    Range(int start, int end) {
-      this.start = start;
-      this.end = end;
-    }
-
-    @Override
-    public String toString() {
-      return "(" + start + ", " + end + ")";
-    }
-  }
-
-  private static Range[] createRanges(int length, int numFiles, Random random) {
-    // generate a number of files with various lengths
-    Range[] ranges = new Range[numFiles];
-    for (int i = 0; i < numFiles; i++) {
-      int start = i == 0 ? 0 : ranges[i-1].end;
-      int end = i == numFiles - 1 ?
-        length :
-        (length/numFiles)*(2*i + 1)/2 + random.nextInt(length/numFiles) + 1;
-      ranges[i] = new Range(start, end);
-    }
-    return ranges;
-  }
-
-  private static void createFiles(int length, int numFiles, Random random)
-    throws IOException {
-    Range[] ranges = createRanges(length, numFiles, random);
-
-    for (int i = 0; i < numFiles; i++) {
-      Path file = new Path(workDir, "test_" + i + ".txt");
-      Writer writer = new OutputStreamWriter(localFs.create(file));
-      Range range = ranges[i];
-      try {
-        for (int j = range.start; j < range.end; j++) {
-          writer.write(Integer.toString(j));
-          writer.write("\n");
-        }
-      } finally {
-        writer.close();
-      }
-    }
-  }
-
-  private static void writeFile(FileSystem fs, Path name,
-                                CompressionCodec codec,
-                                String contents) throws IOException {
-    OutputStream stm;
-    if (codec == null) {
-      stm = fs.create(name);
-    } else {
-      stm = codec.createOutputStream(fs.create(name));
-    }
-    stm.write(contents.getBytes());
-    stm.close();
-  }
-
-  private static List<Text> readSplit(InputFormat<LongWritable,Text> format,
-                                      InputSplit split,
-                                      JobConf job) throws IOException {
-    List<Text> result = new ArrayList<Text>();
-    RecordReader<LongWritable, Text> reader =
-      format.getRecordReader(split, job, voidReporter);
-    LongWritable key = reader.createKey();
-    Text value = reader.createValue();
-    while (reader.next(key, value)) {
-      result.add(value);
-      value = reader.createValue();
-    }
-    reader.close();
-    return result;
-  }
-
-  /**
-   * Test using the gzip codec for reading
-   */
-  @Test(timeout=10000)
-  public void testGzip() throws IOException {
-    JobConf job = new JobConf(defaultConf);
-    CompressionCodec gzip = new GzipCodec();
-    ReflectionUtils.setConf(gzip, job);
-    localFs.delete(workDir, true);
-    writeFile(localFs, new Path(workDir, "part1.txt.gz"), gzip,
-              "the quick\nbrown\nfox jumped\nover\n the lazy\n dog\n");
-    writeFile(localFs, new Path(workDir, "part2.txt.gz"), gzip,
-              "this is a test\nof gzip\n");
-    FileInputFormat.setInputPaths(job, workDir);
-    CombineTextInputFormat format = new CombineTextInputFormat();
-    InputSplit[] splits = format.getSplits(job, 100);
-    assertEquals("compressed splits == 1", 1, splits.length);
-    List<Text> results = readSplit(format, splits[0], job);
-    assertEquals("splits[0] length", 8, results.size());
-
-    final String[] firstList =
-      {"the quick", "brown", "fox jumped", "over", " the lazy", " dog"};
-    final String[] secondList = {"this is a test", "of gzip"};
-    String first = results.get(0).toString();
-    if (first.equals(firstList[0])) {
-      testResults(results, firstList, secondList);
-    } else if (first.equals(secondList[0])) {
-      testResults(results, secondList, firstList);
-    } else {
-      fail("unexpected first token!");
-    }
-  }
-
-  private static void testResults(List<Text> results, String[] first,
-    String[] second) {
-    for (int i = 0; i < first.length; i++) {
-      assertEquals("splits[0]["+i+"]", first[i], results.get(i).toString());
-    }
-    for (int i = 0; i < second.length; i++) {
-      int j = i + first.length;
-      assertEquals("splits[0]["+j+"]", second[i], results.get(j).toString());
-    }
-  }
-}

+ 0 - 334
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapperReducerCleanup.java

@@ -1,334 +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.mapreduce;
-
-import java.io.BufferedWriter;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.util.StringTokenizer;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.LineRecordReader;
-import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.apache.hadoop.mapreduce.lib.reduce.IntSumReducer;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestMapperReducerCleanup {
-
-  static boolean mapCleanup = false;
-  static boolean reduceCleanup = false;
-  static boolean recordReaderCleanup = false;
-  static boolean recordWriterCleanup = false;
-  
-  static void reset() {
-    mapCleanup = false;
-    reduceCleanup = false; 
-    recordReaderCleanup = false;
-    recordWriterCleanup = false;
-  }
-  
-  private static class FailingMapper
-      extends Mapper<LongWritable, Text, LongWritable, Text> {
-
-    /** Map method with different behavior based on the thread id */
-    public void map(LongWritable key, Text val, Context c)
-        throws IOException, InterruptedException {
-      throw new IOException("TestMapperReducerCleanup");
-    }
-
-    protected void cleanup(Context context) 
-        throws IOException, InterruptedException {
-      mapCleanup = true;
-      super.cleanup(context);
-    }
-  }
-
-  private static class TrackingTokenizerMapper 
-  extends Mapper<Object, Text, Text, IntWritable> {
-
-    private final static IntWritable one = new IntWritable(1);
-    private Text word = new Text();
-      
-    public void map(Object key, Text value, Context context
-                    ) throws IOException, InterruptedException {
-      StringTokenizer itr = new StringTokenizer(value.toString());
-      while (itr.hasMoreTokens()) {
-        word.set(itr.nextToken());
-        context.write(word, one);
-      }
-    }
-
-    @SuppressWarnings({ "rawtypes", "unchecked" })
-    @Override
-    protected void cleanup(org.apache.hadoop.mapreduce.Mapper.Context context)
-        throws IOException, InterruptedException {
-      mapCleanup = true;
-      super.cleanup(context);
-    }
-    
-  }
-
-  private static class FailingReducer
-      extends Reducer<LongWritable, Text, LongWritable, LongWritable> {
-
-    public void reduce(LongWritable key, Iterable<Text> vals, Context context)
-        throws IOException, InterruptedException {
-      throw new IOException("TestMapperReducerCleanup");
-    }
-
-    protected void cleanup(Context context) 
-        throws IOException, InterruptedException {
-      reduceCleanup = true;
-      super.cleanup(context);
-    }
-  }
-
-  @SuppressWarnings("rawtypes")
-  private static class TrackingIntSumReducer extends IntSumReducer {
-
-    @SuppressWarnings("unchecked")
-    protected void cleanup(Context context) 
-        throws IOException, InterruptedException {
-      reduceCleanup = true;
-      super.cleanup(context);
-    }
-}
-
-  public static class TrackingTextInputFormat extends TextInputFormat {
-
-    public static class TrackingRecordReader extends LineRecordReader {
-      @Override
-      public synchronized void close() throws IOException {
-        recordReaderCleanup = true;
-        super.close();
-      }
-    }
-
-    @Override
-    public RecordReader<LongWritable, Text> createRecordReader(
-        InputSplit split, TaskAttemptContext context) {
-      return new TrackingRecordReader();
-    }
-  }
-
-  @SuppressWarnings("rawtypes")
-  public static class TrackingTextOutputFormat extends TextOutputFormat {
-    
-    public static class TrackingRecordWriter extends LineRecordWriter {
-
-      public TrackingRecordWriter(DataOutputStream out) {
-        super(out);
-      }
-
-      @Override
-      public synchronized void close(TaskAttemptContext context)
-          throws IOException {
-        recordWriterCleanup = true;
-        super.close(context);
-      }
-
-    }
-    
-    @Override
-    public RecordWriter getRecordWriter(TaskAttemptContext job)
-        throws IOException, InterruptedException {
-      Configuration conf = job.getConfiguration();
-
-      Path file = getDefaultWorkFile(job, "");
-      FileSystem fs = file.getFileSystem(conf);
-      FSDataOutputStream fileOut = fs.create(file, false);
-      
-      return new TrackingRecordWriter(fileOut);
-    }
-  
-  }
-
-
-  /**
-   * Create a single input file in the input directory.
-   * @param dirPath the directory in which the file resides
-   * @param id the file id number
-   * @param numRecords how many records to write to each file.
-   */
-  private void createInputFile(Path dirPath, int id, int numRecords)
-      throws IOException {
-    final String MESSAGE = "This is a line in a file: ";
-
-    Path filePath = new Path(dirPath, "" + id);
-    Configuration conf = new Configuration();
-    FileSystem fs = FileSystem.getLocal(conf);
-
-    OutputStream os = fs.create(filePath);
-    BufferedWriter w = new BufferedWriter(new OutputStreamWriter(os));
-
-    for (int i = 0; i < numRecords; i++) {
-      w.write(MESSAGE + id + " " + i + "\n");
-    }
-
-    w.close();
-  }
-
-  private final String INPUT_DIR = "input";
-  private final String OUTPUT_DIR = "output";
-
-  private Path getInputPath() {
-    String dataDir = System.getProperty("test.build.data");
-    if (null == dataDir) {
-      return new Path(INPUT_DIR);
-    } else {
-      return new Path(new Path(dataDir), INPUT_DIR);
-    }
-  }
-
-  private Path getOutputPath() {
-    String dataDir = System.getProperty("test.build.data");
-    if (null == dataDir) {
-      return new Path(OUTPUT_DIR);
-    } else {
-      return new Path(new Path(dataDir), OUTPUT_DIR);
-    }
-  }
-
-  private Path createInput() throws IOException {
-    Configuration conf = new Configuration();
-    FileSystem fs = FileSystem.getLocal(conf);
-    Path inputPath = getInputPath();
-
-    // Clear the input directory if it exists, first.
-    if (fs.exists(inputPath)) {
-      fs.delete(inputPath, true);
-    }
-
-    // Create an input file
-    createInputFile(inputPath, 0, 10);
-
-    return inputPath;
-  }
-
-  @Test
-  public void testMapCleanup() throws Exception {
-    reset();
-    
-    Job job = Job.getInstance();
-
-    Path inputPath = createInput();
-    Path outputPath = getOutputPath();
-
-    Configuration conf = new Configuration();
-    FileSystem fs = FileSystem.getLocal(conf);
-
-    if (fs.exists(outputPath)) {
-      fs.delete(outputPath, true);
-    }
-
-    job.setMapperClass(FailingMapper.class);
-    job.setInputFormatClass(TrackingTextInputFormat.class);
-    job.setOutputFormatClass(TrackingTextOutputFormat.class);
-    job.setNumReduceTasks(0);
-    FileInputFormat.addInputPath(job, inputPath);
-    FileOutputFormat.setOutputPath(job, outputPath);
-
-    job.waitForCompletion(true);
-
-    Assert.assertTrue(mapCleanup);
-    Assert.assertTrue(recordReaderCleanup);
-    Assert.assertTrue(recordWriterCleanup);
-  }
-
-  @Test
-  public void testReduceCleanup() throws Exception {
-    reset();
-    
-    Job job = Job.getInstance();
-
-    Path inputPath = createInput();
-    Path outputPath = getOutputPath();
-
-    Configuration conf = new Configuration();
-    FileSystem fs = FileSystem.getLocal(conf);
-
-    if (fs.exists(outputPath)) {
-      fs.delete(outputPath, true);
-    }
-
-    job.setMapperClass(TrackingTokenizerMapper.class);
-    job.setReducerClass(FailingReducer.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(IntWritable.class);
-    job.setInputFormatClass(TrackingTextInputFormat.class);
-    job.setOutputFormatClass(TrackingTextOutputFormat.class);
-    job.setNumReduceTasks(1);
-    FileInputFormat.addInputPath(job, inputPath);
-    FileOutputFormat.setOutputPath(job, outputPath);
-
-    job.waitForCompletion(true);
-
-    Assert.assertTrue(mapCleanup);
-    Assert.assertTrue(reduceCleanup);
-    Assert.assertTrue(recordReaderCleanup);
-    Assert.assertTrue(recordWriterCleanup);
-  }
-  
-  @Test
-  public void testJobSuccessCleanup() throws Exception {
-    reset();
-    
-    Job job = Job.getInstance();
-
-    Path inputPath = createInput();
-    Path outputPath = getOutputPath();
-
-    Configuration conf = new Configuration();
-    FileSystem fs = FileSystem.getLocal(conf);
-
-    if (fs.exists(outputPath)) {
-      fs.delete(outputPath, true);
-    }
-
-    job.setMapperClass(TrackingTokenizerMapper.class);
-    job.setReducerClass(TrackingIntSumReducer.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(IntWritable.class);
-    job.setInputFormatClass(TrackingTextInputFormat.class);
-    job.setOutputFormatClass(TrackingTextOutputFormat.class);
-    job.setNumReduceTasks(1);
-    FileInputFormat.addInputPath(job, inputPath);
-    FileOutputFormat.setOutputPath(job, outputPath);
-
-    job.waitForCompletion(true);
-
-    Assert.assertTrue(mapCleanup);
-    Assert.assertTrue(reduceCleanup);
-    Assert.assertTrue(recordReaderCleanup);
-    Assert.assertTrue(recordWriterCleanup);
-  }
-
-}

+ 0 - 186
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java

@@ -1,186 +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.mapreduce.lib.input;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertFalse;
-import static junit.framework.Assert.assertNotNull;
-
-import java.io.IOException;
-import java.util.BitSet;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.MapContext;
-import org.apache.hadoop.mapreduce.MapReduceTestUtil;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.junit.Test;
-
-public class TestCombineSequenceFileInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestCombineSequenceFileInputFormat.class);
-  private static Configuration conf = new Configuration();
-  private static FileSystem localFs = null;
-
-  static {
-    try {
-      conf.set("fs.defaultFS", "file:///");
-      localFs = FileSystem.getLocal(conf);
-    } catch (IOException e) {
-      throw new RuntimeException("init failure", e);
-    }
-  }
-
-  private static Path workDir =
-    new Path(new Path(System.getProperty("test.build.data", "."), "data"),
-             "TestCombineSequenceFileInputFormat");
-
-  @Test(timeout=10000)
-  public void testFormat() throws IOException, InterruptedException {
-    Job job = Job.getInstance(conf);
-
-    Random random = new Random();
-    long seed = random.nextLong();
-    random.setSeed(seed);
-
-    localFs.delete(workDir, true);
-    FileInputFormat.setInputPaths(job, workDir);
-
-    final int length = 10000;
-    final int numFiles = 10;
-
-    // create files with a variety of lengths
-    createFiles(length, numFiles, random, job);
-
-    TaskAttemptContext context = MapReduceTestUtil.
-      createDummyMapTaskAttemptContext(job.getConfiguration());
-    // create a combine split for the files
-    InputFormat<IntWritable,BytesWritable> format =
-      new CombineSequenceFileInputFormat<IntWritable,BytesWritable>();
-    for (int i = 0; i < 3; i++) {
-      int numSplits =
-        random.nextInt(length/(SequenceFile.SYNC_INTERVAL/20)) + 1;
-      LOG.info("splitting: requesting = " + numSplits);
-      List<InputSplit> splits = format.getSplits(job);
-      LOG.info("splitting: got =        " + splits.size());
-
-      // we should have a single split as the length is comfortably smaller than
-      // the block size
-      assertEquals("We got more than one splits!", 1, splits.size());
-      InputSplit split = splits.get(0);
-      assertEquals("It should be CombineFileSplit",
-        CombineFileSplit.class, split.getClass());
-
-      // check the split
-      BitSet bits = new BitSet(length);
-      RecordReader<IntWritable,BytesWritable> reader =
-        format.createRecordReader(split, context);
-      MapContext<IntWritable,BytesWritable,IntWritable,BytesWritable> mcontext =
-        new MapContextImpl<IntWritable,BytesWritable,IntWritable,BytesWritable>(job.getConfiguration(),
-        context.getTaskAttemptID(), reader, null, null,
-        MapReduceTestUtil.createDummyReporter(), split);
-      reader.initialize(split, mcontext);
-      assertEquals("reader class is CombineFileRecordReader.",
-        CombineFileRecordReader.class, reader.getClass());
-
-      try {
-        while (reader.nextKeyValue()) {
-          IntWritable key = reader.getCurrentKey();
-          BytesWritable value = reader.getCurrentValue();
-          assertNotNull("Value should not be null.", value);
-          final int k = key.get();
-          LOG.debug("read " + k);
-          assertFalse("Key in multiple partitions.", bits.get(k));
-          bits.set(k);
-        }
-      } finally {
-        reader.close();
-      }
-      assertEquals("Some keys in no partition.", length, bits.cardinality());
-    }
-  }
-
-
-  private static class Range {
-    private final int start;
-    private final int end;
-
-    Range(int start, int end) {
-      this.start = start;
-      this.end = end;
-    }
-
-    @Override
-    public String toString() {
-      return "(" + start + ", " + end + ")";
-    }
-  }
-
-  private static Range[] createRanges(int length, int numFiles, Random random) {
-    // generate a number of files with various lengths
-    Range[] ranges = new Range[numFiles];
-    for (int i = 0; i < numFiles; i++) {
-      int start = i == 0 ? 0 : ranges[i-1].end;
-      int end = i == numFiles - 1 ?
-        length :
-        (length/numFiles)*(2*i + 1)/2 + random.nextInt(length/numFiles) + 1;
-      ranges[i] = new Range(start, end);
-    }
-    return ranges;
-  }
-
-  private static void createFiles(int length, int numFiles, Random random,
-    Job job) throws IOException {
-    Range[] ranges = createRanges(length, numFiles, random);
-
-    for (int i = 0; i < numFiles; i++) {
-      Path file = new Path(workDir, "test_" + i + ".seq");
-      // create a file with length entries
-      @SuppressWarnings("deprecation")
-      SequenceFile.Writer writer =
-        SequenceFile.createWriter(localFs, job.getConfiguration(), file,
-                                  IntWritable.class, BytesWritable.class);
-      Range range = ranges[i];
-      try {
-        for (int j = range.start; j < range.end; j++) {
-          IntWritable key = new IntWritable(j);
-          byte[] data = new byte[random.nextInt(10)];
-          random.nextBytes(data);
-          BytesWritable value = new BytesWritable(data);
-          writer.append(key, value);
-        }
-      } finally {
-        writer.close();
-      }
-    }
-  }
-}

+ 0 - 267
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java

@@ -1,267 +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.mapreduce.lib.input;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertFalse;
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.fail;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.MapContext;
-import org.apache.hadoop.mapreduce.MapReduceTestUtil;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.task.MapContextImpl;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.junit.Test;
-
-public class TestCombineTextInputFormat {
-  private static final Log LOG =
-    LogFactory.getLog(TestCombineTextInputFormat.class);
-
-  private static Configuration defaultConf = new Configuration();
-  private static FileSystem localFs = null;
-
-  static {
-    try {
-      defaultConf.set("fs.defaultFS", "file:///");
-      localFs = FileSystem.getLocal(defaultConf);
-    } catch (IOException e) {
-      throw new RuntimeException("init failure", e);
-    }
-  }
-
-  private static Path workDir =
-    new Path(new Path(System.getProperty("test.build.data", "."), "data"),
-             "TestCombineTextInputFormat");
-
-  @Test(timeout=10000)
-  public void testFormat() throws Exception {
-    Job job = Job.getInstance(new Configuration(defaultConf));
-
-    Random random = new Random();
-    long seed = random.nextLong();
-    LOG.info("seed = " + seed);
-    random.setSeed(seed);
-
-    localFs.delete(workDir, true);
-    FileInputFormat.setInputPaths(job, workDir);
-
-    final int length = 10000;
-    final int numFiles = 10;
-
-    // create files with various lengths
-    createFiles(length, numFiles, random);
-
-    // create a combined split for the files
-    CombineTextInputFormat format = new CombineTextInputFormat();
-    for (int i = 0; i < 3; i++) {
-      int numSplits = random.nextInt(length/20) + 1;
-      LOG.info("splitting: requesting = " + numSplits);
-      List<InputSplit> splits = format.getSplits(job);
-      LOG.info("splitting: got =        " + splits.size());
-
-      // we should have a single split as the length is comfortably smaller than
-      // the block size
-      assertEquals("We got more than one splits!", 1, splits.size());
-      InputSplit split = splits.get(0);
-      assertEquals("It should be CombineFileSplit",
-        CombineFileSplit.class, split.getClass());
-
-      // check the split
-      BitSet bits = new BitSet(length);
-      LOG.debug("split= " + split);
-      TaskAttemptContext context = MapReduceTestUtil.
-        createDummyMapTaskAttemptContext(job.getConfiguration());
-      RecordReader<LongWritable, Text> reader =
-        format.createRecordReader(split, context);
-      assertEquals("reader class is CombineFileRecordReader.",
-        CombineFileRecordReader.class, reader.getClass());
-      MapContext<LongWritable,Text,LongWritable,Text> mcontext =
-        new MapContextImpl<LongWritable,Text,LongWritable,Text>(job.getConfiguration(),
-        context.getTaskAttemptID(), reader, null, null,
-        MapReduceTestUtil.createDummyReporter(), split);
-      reader.initialize(split, mcontext);
-
-      try {
-        int count = 0;
-        while (reader.nextKeyValue()) {
-          LongWritable key = reader.getCurrentKey();
-          assertNotNull("Key should not be null.", key);
-          Text value = reader.getCurrentValue();
-          final int v = Integer.parseInt(value.toString());
-          LOG.debug("read " + v);
-          assertFalse("Key in multiple partitions.", bits.get(v));
-          bits.set(v);
-          count++;
-        }
-        LOG.debug("split=" + split + " count=" + count);
-      } finally {
-        reader.close();
-      }
-      assertEquals("Some keys in no partition.", length, bits.cardinality());
-    }
-  }
-
-  private static class Range {
-    private final int start;
-    private final int end;
-
-    Range(int start, int end) {
-      this.start = start;
-      this.end = end;
-    }
-
-    @Override
-    public String toString() {
-      return "(" + start + ", " + end + ")";
-    }
-  }
-
-  private static Range[] createRanges(int length, int numFiles, Random random) {
-    // generate a number of files with various lengths
-    Range[] ranges = new Range[numFiles];
-    for (int i = 0; i < numFiles; i++) {
-      int start = i == 0 ? 0 : ranges[i-1].end;
-      int end = i == numFiles - 1 ?
-        length :
-        (length/numFiles)*(2*i + 1)/2 + random.nextInt(length/numFiles) + 1;
-      ranges[i] = new Range(start, end);
-    }
-    return ranges;
-  }
-
-  private static void createFiles(int length, int numFiles, Random random)
-    throws IOException {
-    Range[] ranges = createRanges(length, numFiles, random);
-
-    for (int i = 0; i < numFiles; i++) {
-      Path file = new Path(workDir, "test_" + i + ".txt");
-      Writer writer = new OutputStreamWriter(localFs.create(file));
-      Range range = ranges[i];
-      try {
-        for (int j = range.start; j < range.end; j++) {
-          writer.write(Integer.toString(j));
-          writer.write("\n");
-        }
-      } finally {
-        writer.close();
-      }
-    }
-  }
-
-  private static void writeFile(FileSystem fs, Path name,
-                                CompressionCodec codec,
-                                String contents) throws IOException {
-    OutputStream stm;
-    if (codec == null) {
-      stm = fs.create(name);
-    } else {
-      stm = codec.createOutputStream(fs.create(name));
-    }
-    stm.write(contents.getBytes());
-    stm.close();
-  }
-
-  private static List<Text> readSplit(InputFormat<LongWritable,Text> format,
-    InputSplit split, Job job) throws IOException, InterruptedException {
-    List<Text> result = new ArrayList<Text>();
-    Configuration conf = job.getConfiguration();
-    TaskAttemptContext context = MapReduceTestUtil.
-      createDummyMapTaskAttemptContext(conf);
-    RecordReader<LongWritable, Text> reader = format.createRecordReader(split,
-      MapReduceTestUtil.createDummyMapTaskAttemptContext(conf));
-    MapContext<LongWritable,Text,LongWritable,Text> mcontext =
-      new MapContextImpl<LongWritable,Text,LongWritable,Text>(conf,
-      context.getTaskAttemptID(), reader, null, null,
-      MapReduceTestUtil.createDummyReporter(),
-      split);
-    reader.initialize(split, mcontext);
-    while (reader.nextKeyValue()) {
-      result.add(new Text(reader.getCurrentValue()));
-    }
-    return result;
-  }
-
-  /**
-   * Test using the gzip codec for reading
-   */
-  @Test(timeout=10000)
-  public void testGzip() throws IOException, InterruptedException {
-    Configuration conf = new Configuration(defaultConf);
-    CompressionCodec gzip = new GzipCodec();
-    ReflectionUtils.setConf(gzip, conf);
-    localFs.delete(workDir, true);
-    writeFile(localFs, new Path(workDir, "part1.txt.gz"), gzip,
-              "the quick\nbrown\nfox jumped\nover\n the lazy\n dog\n");
-    writeFile(localFs, new Path(workDir, "part2.txt.gz"), gzip,
-              "this is a test\nof gzip\n");
-    Job job = Job.getInstance(conf);
-    FileInputFormat.setInputPaths(job, workDir);
-    CombineTextInputFormat format = new CombineTextInputFormat();
-    List<InputSplit> splits = format.getSplits(job);
-    assertEquals("compressed splits == 1", 1, splits.size());
-    List<Text> results = readSplit(format, splits.get(0), job);
-    assertEquals("splits[0] length", 8, results.size());
-
-    final String[] firstList =
-      {"the quick", "brown", "fox jumped", "over", " the lazy", " dog"};
-    final String[] secondList = {"this is a test", "of gzip"};
-    String first = results.get(0).toString();
-    if (first.equals(firstList[0])) {
-      testResults(results, firstList, secondList);
-    } else if (first.equals(secondList[0])) {
-      testResults(results, secondList, firstList);
-    } else {
-      fail("unexpected first token!");
-    }
-  }
-
-  private static void testResults(List<Text> results, String[] first,
-    String[] second) {
-    for (int i = 0; i < first.length; i++) {
-      assertEquals("splits[0]["+i+"]", first[i], results.get(i).toString());
-    }
-    for (int i = 0; i < second.length; i++) {
-      int j = i + first.length;
-      assertEquals("splits[0]["+j+"]", second[i], results.get(j).toString());
-    }
-  }
-}

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

@@ -98,10 +98,6 @@ Release 2.0.5-beta - UNRELEASED
     YARN-561. Modified NodeManager to set key information into the environment
     YARN-561. Modified NodeManager to set key information into the environment
     of every container that it launches. (Xuan Gong via vinodkv)
     of every container that it launches. (Xuan Gong via vinodkv)
 
 
-    YARN-579. Stop setting the Application Token in the AppMaster env, in
-    favour of the copy present in the container token field. 
-    (Vinod Kumar Vavilapalli via sseth)
-
   NEW FEATURES
   NEW FEATURES
 
 
     YARN-482. FS: Extend SchedulingMode to intermediate queues. 
     YARN-482. FS: Extend SchedulingMode to intermediate queues. 
@@ -181,12 +177,6 @@ Release 2.0.5-beta - UNRELEASED
     YARN-581. Added a test to verify that app delegation tokens are restored
     YARN-581. Added a test to verify that app delegation tokens are restored
     after RM restart. (Jian He via vinodkv)
     after RM restart. (Jian He via vinodkv)
 
 
-    YARN-577. Add application-progress also to ApplicationReport. (Hitesh Shah
-    via vinodkv)
-
-    YARN-595. Refactor fair scheduler to use common Resources. (Sandy Ryza
-    via tomwhite)
-
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -300,12 +290,6 @@ Release 2.0.5-beta - UNRELEASED
     YARN-549. YarnClient.submitApplication should wait for application to be
     YARN-549. YarnClient.submitApplication should wait for application to be
     accepted by the RM (Zhijie Shen via bikas)
     accepted by the RM (Zhijie Shen via bikas)
 
 
-    YARN-605. Fix failing unit test in TestNMWebServices when versionInfo has
-    parantheses like when running on a git checkout. (Hitesh Shah via vinodkv)
-
-    YARN-289. Fair scheduler allows reservations that won't fit on node.
-    (Sandy Ryza via tomwhite)
-
 Release 2.0.4-alpha - UNRELEASED
 Release 2.0.4-alpha - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java

@@ -29,6 +29,10 @@ import org.apache.hadoop.util.Shell;
  */
  */
 public interface ApplicationConstants {
 public interface ApplicationConstants {
 
 
+  // TODO: They say tokens via env isn't good.
+  public static final String APPLICATION_MASTER_TOKEN_ENV_NAME =
+    "AppMasterTokenEnv";
+
   // TODO: They say tokens via env isn't good.
   // TODO: They say tokens via env isn't good.
   public static final String APPLICATION_CLIENT_SECRET_ENV_NAME =
   public static final String APPLICATION_CLIENT_SECRET_ENV_NAME =
     "AppClientSecretEnv";
     "AppClientSecretEnv";

+ 0 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java

@@ -253,16 +253,4 @@ public interface ApplicationReport {
   @Private
   @Private
   @Unstable
   @Unstable
   void setApplicationResourceUsageReport(ApplicationResourceUsageReport appResources);
   void setApplicationResourceUsageReport(ApplicationResourceUsageReport appResources);
-
-  /**
-   * Get the application's progress ( range 0.0 to 1.0 )
-   * @return application's progress
-   */
-  @Public
-  @Stable
-  float getProgress();
-
-  @Private
-  @Unstable
-  void setProgress(float progress);
 }
 }

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

@@ -213,12 +213,6 @@ implements ApplicationReport {
     return convertFromProtoFormat(p.getFinalApplicationStatus());
     return convertFromProtoFormat(p.getFinalApplicationStatus());
   }
   }
 
 
-  @Override
-  public float getProgress() {
-    ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
-    return p.getProgress();
-  }
-
   @Override
   @Override
   public void setApplicationId(ApplicationId applicationId) {
   public void setApplicationId(ApplicationId applicationId) {
     maybeInitBuilder();
     maybeInitBuilder();
@@ -351,12 +345,6 @@ implements ApplicationReport {
     builder.setFinalApplicationStatus(convertToProtoFormat(finishState));
     builder.setFinalApplicationStatus(convertToProtoFormat(finishState));
   }
   }
 
 
-  @Override
-  public void setProgress(float progress) {
-    maybeInitBuilder();
-    builder.setProgress(progress);
-  }
-
   @Override
   @Override
   public ApplicationReportProto getProto() {
   public ApplicationReportProto getProto() {
     mergeLocalToProto();
     mergeLocalToProto();

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -162,7 +162,6 @@ message ApplicationReportProto {
   optional ApplicationResourceUsageReportProto app_resource_Usage = 16;
   optional ApplicationResourceUsageReportProto app_resource_Usage = 16;
   optional string originalTrackingUrl = 17;
   optional string originalTrackingUrl = 17;
   optional ApplicationAttemptIdProto currentApplicationAttemptId = 18;
   optional ApplicationAttemptIdProto currentApplicationAttemptId = 18;
-  optional float progress = 19;
 }
 }
 
 
 enum NodeStateProto {
 enum NodeStateProto {

+ 22 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java

@@ -33,9 +33,13 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
@@ -111,7 +115,24 @@ public class AMRMClientImpl extends AbstractService implements AMRMClient {
       throw new YarnException(e);
       throw new YarnException(e);
     }
     }
 
 
-    // CurrentUser should already have AMToken loaded.
+    if (UserGroupInformation.isSecurityEnabled()) {
+      String tokenURLEncodedStr = System.getenv().get(
+          ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME);
+      Token<? extends TokenIdentifier> token = new Token<TokenIdentifier>();
+
+      try {
+        token.decodeFromUrlString(tokenURLEncodedStr);
+      } catch (IOException e) {
+        throw new YarnException(e);
+      }
+
+      SecurityUtil.setTokenService(token, rmAddress);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("AppMasterToken is " + token);
+      }
+      currentUser.addToken(token);
+    }
+
     rmClient = currentUser.doAs(new PrivilegedAction<AMRMProtocol>() {
     rmClient = currentUser.doAs(new PrivilegedAction<AMRMProtocol>() {
       @Override
       @Override
       public AMRMProtocol run() {
       public AMRMProtocol run() {

+ 3 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.client.cli;
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
-import java.text.DecimalFormat;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
@@ -35,7 +34,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 
 
 public class ApplicationCLI extends YarnCLI {
 public class ApplicationCLI extends YarnCLI {
   private static final String APPLICATIONS_PATTERN =
   private static final String APPLICATIONS_PATTERN =
-    "%30s\t%20s\t%10s\t%10s\t%18s\t%18s\t%15s\t%35s" +
+    "%30s\t%20s\t%10s\t%10s\t%18s\t%18s\t%35s" +
     System.getProperty("line.separator");
     System.getProperty("line.separator");
 
 
   public static void main(String[] args) throws Exception {
   public static void main(String[] args) throws Exception {
@@ -99,15 +98,12 @@ public class ApplicationCLI extends YarnCLI {
     writer.println("Total Applications:" + appsReport.size());
     writer.println("Total Applications:" + appsReport.size());
     writer.printf(APPLICATIONS_PATTERN, "Application-Id",
     writer.printf(APPLICATIONS_PATTERN, "Application-Id",
         "Application-Name", "User", "Queue", "State", "Final-State",
         "Application-Name", "User", "Queue", "State", "Final-State",
-        "Progress", "Tracking-URL");
+        "Tracking-URL");
     for (ApplicationReport appReport : appsReport) {
     for (ApplicationReport appReport : appsReport) {
-      DecimalFormat formatter = new DecimalFormat("###.##%");
-      String progress = formatter.format(appReport.getProgress());
       writer.printf(APPLICATIONS_PATTERN, appReport.getApplicationId(),
       writer.printf(APPLICATIONS_PATTERN, appReport.getApplicationId(),
           appReport.getName(), appReport.getUser(), appReport.getQueue(),
           appReport.getName(), appReport.getUser(), appReport.getQueue(),
           appReport.getYarnApplicationState(), appReport
           appReport.getYarnApplicationState(), appReport
-              .getFinalApplicationStatus(),
-          progress, appReport.getOriginalTrackingUrl());
+              .getFinalApplicationStatus(), appReport.getOriginalTrackingUrl());
     }
     }
     writer.flush();
     writer.flush();
   }
   }
@@ -151,10 +147,6 @@ public class ApplicationCLI extends YarnCLI {
       appReportStr.println(appReport.getStartTime());
       appReportStr.println(appReport.getStartTime());
       appReportStr.print("\tFinish-Time : ");
       appReportStr.print("\tFinish-Time : ");
       appReportStr.println(appReport.getFinishTime());
       appReportStr.println(appReport.getFinishTime());
-      appReportStr.print("\tProgress : ");
-      DecimalFormat formatter = new DecimalFormat("###.##%");
-      String progress = formatter.format(appReport.getProgress());
-      appReportStr.println(progress);
       appReportStr.print("\tState : ");
       appReportStr.print("\tState : ");
       appReportStr.println(appReport.getYarnApplicationState());
       appReportStr.println(appReport.getYarnApplicationState());
       appReportStr.print("\tFinal-State : ");
       appReportStr.print("\tFinal-State : ");

+ 4 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java

@@ -76,7 +76,7 @@ public class TestYarnCLI {
         applicationId, BuilderUtils.newApplicationAttemptId(applicationId, 1),
         applicationId, BuilderUtils.newApplicationAttemptId(applicationId, 1),
         "user", "queue", "appname", "host", 124, null,
         "user", "queue", "appname", "host", 124, null,
         YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
         YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
-        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f);
+        FinalApplicationStatus.SUCCEEDED, null, "N/A");
     when(client.getApplicationReport(any(ApplicationId.class))).thenReturn(
     when(client.getApplicationReport(any(ApplicationId.class))).thenReturn(
         newApplicationReport);
         newApplicationReport);
     int result = cli.run(new String[] { "-status", applicationId.toString() });
     int result = cli.run(new String[] { "-status", applicationId.toString() });
@@ -91,7 +91,6 @@ public class TestYarnCLI {
     pw.println("\tQueue : queue");
     pw.println("\tQueue : queue");
     pw.println("\tStart-Time : 0");
     pw.println("\tStart-Time : 0");
     pw.println("\tFinish-Time : 0");
     pw.println("\tFinish-Time : 0");
-    pw.println("\tProgress : 53.79%");
     pw.println("\tState : FINISHED");
     pw.println("\tState : FINISHED");
     pw.println("\tFinal-State : SUCCEEDED");
     pw.println("\tFinal-State : SUCCEEDED");
     pw.println("\tTracking-URL : N/A");
     pw.println("\tTracking-URL : N/A");
@@ -112,7 +111,7 @@ public class TestYarnCLI {
         applicationId, BuilderUtils.newApplicationAttemptId(applicationId, 1),
         applicationId, BuilderUtils.newApplicationAttemptId(applicationId, 1),
         "user", "queue", "appname", "host", 124, null,
         "user", "queue", "appname", "host", 124, null,
         YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
         YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
-        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f);
+        FinalApplicationStatus.SUCCEEDED, null, "N/A");
     List<ApplicationReport> applicationReports = new ArrayList<ApplicationReport>();
     List<ApplicationReport> applicationReports = new ArrayList<ApplicationReport>();
     applicationReports.add(newApplicationReport);
     applicationReports.add(newApplicationReport);
     when(client.getApplicationList()).thenReturn(applicationReports);
     when(client.getApplicationList()).thenReturn(applicationReports);
@@ -125,12 +124,10 @@ public class TestYarnCLI {
     pw.println("Total Applications:1");
     pw.println("Total Applications:1");
     pw.print("                Application-Id\t    Application-Name");
     pw.print("                Application-Id\t    Application-Name");
     pw.print("\t      User\t     Queue\t             State\t       ");
     pw.print("\t      User\t     Queue\t             State\t       ");
-    pw.print("Final-State\t       Progress");
-    pw.println("\t                       Tracking-URL");
+    pw.println("Final-State\t                       Tracking-URL");
     pw.print("         application_1234_0005\t             ");
     pw.print("         application_1234_0005\t             ");
     pw.print("appname\t      user\t     queue\t          FINISHED\t         ");
     pw.print("appname\t      user\t     queue\t          FINISHED\t         ");
-    pw.print("SUCCEEDED\t         53.79%");
-    pw.println("\t                                N/A");
+    pw.println("SUCCEEDED\t                                N/A");
     pw.close();
     pw.close();
     String appsReportStr = baos.toString("UTF-8");
     String appsReportStr = baos.toString("UTF-8");
     Assert.assertEquals(appsReportStr, sysOutStream.toString());
     Assert.assertEquals(appsReportStr, sysOutStream.toString());

+ 1 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java

@@ -333,8 +333,7 @@ public class BuilderUtils {
       ClientToken clientToken, YarnApplicationState state, String diagnostics,
       ClientToken clientToken, YarnApplicationState state, String diagnostics,
       String url, long startTime, long finishTime,
       String url, long startTime, long finishTime,
       FinalApplicationStatus finalStatus,
       FinalApplicationStatus finalStatus,
-      ApplicationResourceUsageReport appResources, String origTrackingUrl,
-      float progress) {
+      ApplicationResourceUsageReport appResources, String origTrackingUrl) {
     ApplicationReport report = recordFactory
     ApplicationReport report = recordFactory
         .newRecordInstance(ApplicationReport.class);
         .newRecordInstance(ApplicationReport.class);
     report.setApplicationId(applicationId);
     report.setApplicationId(applicationId);
@@ -353,7 +352,6 @@ public class BuilderUtils {
     report.setFinalApplicationStatus(finalStatus);
     report.setFinalApplicationStatus(finalStatus);
     report.setApplicationResourceUsageReport(appResources);
     report.setApplicationResourceUsageReport(appResources);
     report.setOriginalTrackingUrl(origTrackingUrl);
     report.setOriginalTrackingUrl(origTrackingUrl);
-    report.setProgress(progress);
     return report;
     return report;
   }
   }
 
 

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java

@@ -85,10 +85,5 @@ public class WebServicesTestUtils {
         got.contains(expected));
         got.contains(expected));
   }
   }
 
 
-  public static void checkStringEqual(String print, String expected, String got) {
-    assertTrue(
-        print + " is not equal, got: " + got + " expected: " + expected,
-        got.equals(expected));
-  }
 
 
 }
 }

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

@@ -360,14 +360,14 @@ public class TestNMWebServices extends JerseyTest {
 
 
     WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn",
     WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn",
         VersionInfo.getDate(), hadoopVersionBuiltOn);
         VersionInfo.getDate(), hadoopVersionBuiltOn);
-    WebServicesTestUtils.checkStringEqual("hadoopBuildVersion",
+    WebServicesTestUtils.checkStringMatch("hadoopBuildVersion",
         VersionInfo.getBuildVersion(), hadoopBuildVersion);
         VersionInfo.getBuildVersion(), hadoopBuildVersion);
     WebServicesTestUtils.checkStringMatch("hadoopVersion",
     WebServicesTestUtils.checkStringMatch("hadoopVersion",
         VersionInfo.getVersion(), hadoopVersion);
         VersionInfo.getVersion(), hadoopVersion);
 
 
     WebServicesTestUtils.checkStringMatch("resourceManagerVersionBuiltOn",
     WebServicesTestUtils.checkStringMatch("resourceManagerVersionBuiltOn",
         YarnVersionInfo.getDate(), resourceManagerVersionBuiltOn);
         YarnVersionInfo.getDate(), resourceManagerVersionBuiltOn);
-    WebServicesTestUtils.checkStringEqual("resourceManagerBuildVersion",
+    WebServicesTestUtils.checkStringMatch("resourceManagerBuildVersion",
         YarnVersionInfo.getBuildVersion(), resourceManagerBuildVersion);
         YarnVersionInfo.getBuildVersion(), resourceManagerBuildVersion);
     WebServicesTestUtils.checkStringMatch("resourceManagerVersion",
     WebServicesTestUtils.checkStringMatch("resourceManagerVersion",
         YarnVersionInfo.getVersion(), resourceManagerVersion);
         YarnVersionInfo.getVersion(), resourceManagerVersion);

+ 10 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java

@@ -204,7 +204,7 @@ public class AMLauncher implements Runnable {
 
 
       ApplicationTokenIdentifier id = new ApplicationTokenIdentifier(
       ApplicationTokenIdentifier id = new ApplicationTokenIdentifier(
           application.getAppAttemptId());
           application.getAppAttemptId());
-      Token<ApplicationTokenIdentifier> appMasterToken =
+      Token<ApplicationTokenIdentifier> token =
           new Token<ApplicationTokenIdentifier>(id,
           new Token<ApplicationTokenIdentifier>(id,
               this.rmContext.getApplicationTokenSecretManager());
               this.rmContext.getApplicationTokenSecretManager());
       InetSocketAddress serviceAddr = conf.getSocketAddr(
       InetSocketAddress serviceAddr = conf.getSocketAddr(
@@ -212,11 +212,16 @@ public class AMLauncher implements Runnable {
           YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
           YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
           YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
           YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
       // normally the client should set the service after acquiring the token,
       // normally the client should set the service after acquiring the token,
-      // but this token is directly provided to the AMs
-      SecurityUtil.setTokenService(appMasterToken, serviceAddr);
+      // but this token is directly provided to the tasks
+      SecurityUtil.setTokenService(token, serviceAddr);
+      String appMasterTokenEncoded = token.encodeToUrlString();
+      LOG.debug("Putting appMaster token in env : " + token);
+      environment.put(
+          ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME,
+          appMasterTokenEncoded);
 
 
-      // Add the ApplicationMaster token
-      credentials.addToken(appMasterToken.getService(), appMasterToken);
+      // Add the RM token
+      credentials.addToken(token.getService(), token);
       DataOutputBuffer dob = new DataOutputBuffer();
       DataOutputBuffer dob = new DataOutputBuffer();
       credentials.writeTokenStorageToStream(dob);
       credentials.writeTokenStorageToStream(dob);
       container.setContainerTokens(
       container.setContainerTokens(

+ 0 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java

@@ -208,14 +208,4 @@ public class Resources {
       Resource lhs, Resource rhs) {
       Resource lhs, Resource rhs) {
     return resourceCalculator.compare(clusterResource, lhs, rhs) >= 0 ? lhs : rhs;
     return resourceCalculator.compare(clusterResource, lhs, rhs) >= 0 ? lhs : rhs;
   }
   }
-  
-  public static boolean fitsIn(Resource smaller, Resource bigger) {
-    return smaller.getMemory() <= bigger.getMemory() &&
-        smaller.getVirtualCores() <= bigger.getVirtualCores();
-  }
-  
-  public static Resource componentwiseMin(Resource lhs, Resource rhs) {
-    return createResource(Math.min(lhs.getMemory(), rhs.getMemory()),
-        Math.min(lhs.getVirtualCores(), rhs.getVirtualCores()));
-  }
 }
 }

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

@@ -437,7 +437,6 @@ public class RMAppImpl implements RMApp, Recoverable {
           DUMMY_APPLICATION_RESOURCE_USAGE_REPORT;
           DUMMY_APPLICATION_RESOURCE_USAGE_REPORT;
       FinalApplicationStatus finishState = getFinalApplicationStatus();
       FinalApplicationStatus finishState = getFinalApplicationStatus();
       String diags = UNAVAILABLE;
       String diags = UNAVAILABLE;
-      float progress = 0.0f;
       if (allowAccess) {
       if (allowAccess) {
         if (this.currentAttempt != null) {
         if (this.currentAttempt != null) {
           currentApplicationAttemptId = this.currentAttempt.getAppAttemptId();
           currentApplicationAttemptId = this.currentAttempt.getAppAttemptId();
@@ -447,8 +446,8 @@ public class RMAppImpl implements RMApp, Recoverable {
           host = this.currentAttempt.getHost();
           host = this.currentAttempt.getHost();
           rpcPort = this.currentAttempt.getRpcPort();
           rpcPort = this.currentAttempt.getRpcPort();
           appUsageReport = currentAttempt.getApplicationResourceUsageReport();
           appUsageReport = currentAttempt.getApplicationResourceUsageReport();
-          progress = currentAttempt.getProgress();
         }
         }
+
         diags = this.diagnostics.toString();
         diags = this.diagnostics.toString();
       }
       }
 
 
@@ -463,7 +462,7 @@ public class RMAppImpl implements RMApp, Recoverable {
           this.name, host, rpcPort, clientToken,
           this.name, host, rpcPort, clientToken,
           createApplicationState(this.stateMachine.getCurrentState()), diags,
           createApplicationState(this.stateMachine.getCurrentState()), diags,
           trackingUrl, this.startTime, this.finishTime, finishState,
           trackingUrl, this.startTime, this.finishTime, finishState,
-          appUsageReport, origTrackingUrl, progress);
+          appUsageReport, origTrackingUrl);
     } finally {
     } finally {
       this.readLock.unlock();
       this.readLock.unlock();
     }
     }

+ 26 - 45
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

@@ -35,7 +35,6 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
@@ -46,9 +45,6 @@ import org.apache.hadoop.yarn.util.BuilderUtils;
 @Private
 @Private
 @Unstable
 @Unstable
 public class AppSchedulable extends Schedulable {
 public class AppSchedulable extends Schedulable {
-  private static final DefaultResourceCalculator RESOURCE_CALCULATOR
-    = new DefaultResourceCalculator();
-  
   private FairScheduler scheduler;
   private FairScheduler scheduler;
   private FSSchedulerApp app;
   private FSSchedulerApp app;
   private Resource demand = Resources.createResource(0);
   private Resource demand = Resources.createResource(0);
@@ -184,15 +180,15 @@ public class AppSchedulable extends Schedulable {
    * update relevant bookeeping. This dispatches ro relevant handlers
    * update relevant bookeeping. This dispatches ro relevant handlers
    * in the {@link FSSchedulerNode} and {@link SchedulerApp} classes.
    * in the {@link FSSchedulerNode} and {@link SchedulerApp} classes.
    */
    */
-  private void reserve(Priority priority, FSSchedulerNode node,
-      Container container, boolean alreadyReserved) {
+  private void reserve(FSSchedulerApp application, Priority priority,
+      FSSchedulerNode node, Container container, boolean alreadyReserved) {
     LOG.info("Making reservation: node=" + node.getHostName() +
     LOG.info("Making reservation: node=" + node.getHostName() +
                                  " app_id=" + app.getApplicationId());
                                  " app_id=" + app.getApplicationId());
     if (!alreadyReserved) {
     if (!alreadyReserved) {
-      getMetrics().reserveResource(app.getUser(), container.getResource());
-      RMContainer rmContainer = app.reserve(node, priority, null,
+      getMetrics().reserveResource(application.getUser(), container.getResource());
+      RMContainer rmContainer = application.reserve(node, priority, null,
           container);
           container);
-      node.reserveResource(app, priority, rmContainer);
+      node.reserveResource(application, priority, rmContainer);
       getMetrics().reserveResource(app.getUser(),
       getMetrics().reserveResource(app.getUser(),
           container.getResource());
           container.getResource());
       scheduler.getRootQueueMetrics().reserveResource(app.getUser(),
       scheduler.getRootQueueMetrics().reserveResource(app.getUser(),
@@ -201,24 +197,25 @@ public class AppSchedulable extends Schedulable {
 
 
     else {
     else {
       RMContainer rmContainer = node.getReservedContainer();
       RMContainer rmContainer = node.getReservedContainer();
-      app.reserve(node, priority, rmContainer, container);
-      node.reserveResource(app, priority, rmContainer);
+      application.reserve(node, priority, rmContainer, container);
+      node.reserveResource(application, priority, rmContainer);
     }
     }
   }
   }
 
 
   /**
   /**
-   * Remove the reservation on {@code node} at the given
+   * Remove the reservation on {@code node} for {@ application} at the given
    * {@link Priority}. This dispatches to the SchedulerApp and SchedulerNode
    * {@link Priority}. This dispatches to the SchedulerApp and SchedulerNode
    * handlers for an unreservation.
    * handlers for an unreservation.
    */
    */
-  public void unreserve(Priority priority, FSSchedulerNode node) {
+  private void unreserve(FSSchedulerApp application, Priority priority,
+      FSSchedulerNode node) {
     RMContainer rmContainer = node.getReservedContainer();
     RMContainer rmContainer = node.getReservedContainer();
-    app.unreserve(node, priority);
-    node.unreserveResource(app);
+    application.unreserve(node, priority);
+    node.unreserveResource(application);
     getMetrics().unreserveResource(
     getMetrics().unreserveResource(
-        app.getUser(), rmContainer.getContainer().getResource());
+        application.getUser(), rmContainer.getContainer().getResource());
     scheduler.getRootQueueMetrics().unreserveResource(
     scheduler.getRootQueueMetrics().unreserveResource(
-        app.getUser(), rmContainer.getContainer().getResource());
+        application.getUser(), rmContainer.getContainer().getResource());
   }
   }
 
 
   /**
   /**
@@ -227,8 +224,8 @@ public class AppSchedulable extends Schedulable {
    * sure the particular request should be facilitated by this node.
    * sure the particular request should be facilitated by this node.
    */
    */
   private Resource assignContainer(FSSchedulerNode node,
   private Resource assignContainer(FSSchedulerNode node,
-      Priority priority, ResourceRequest request, NodeType type,
-      boolean reserved) {
+      FSSchedulerApp application, Priority priority,
+      ResourceRequest request, NodeType type, boolean reserved) {
 
 
     // How much does this request need?
     // How much does this request need?
     Resource capability = request.getCapability();
     Resource capability = request.getCapability();
@@ -240,7 +237,7 @@ public class AppSchedulable extends Schedulable {
     if (reserved) {
     if (reserved) {
       container = node.getReservedContainer().getContainer();
       container = node.getReservedContainer().getContainer();
     } else {
     } else {
-      container = createContainer(app, node, capability, priority);
+      container = createContainer(application, node, capability, priority);
     }
     }
 
 
     // Can we allocate a container on this node?
     // Can we allocate a container on this node?
@@ -250,12 +247,9 @@ public class AppSchedulable extends Schedulable {
     if (availableContainers > 0) {
     if (availableContainers > 0) {
       // Inform the application of the new container for this request
       // Inform the application of the new container for this request
       RMContainer allocatedContainer =
       RMContainer allocatedContainer =
-          app.allocate(type, node, priority, request, container);
+          application.allocate(type, node, priority, request, container);
       if (allocatedContainer == null) {
       if (allocatedContainer == null) {
         // Did the application need this resource?
         // Did the application need this resource?
-        if (reserved) {
-          unreserve(priority, node);
-        }
         return Resources.none();
         return Resources.none();
       }
       }
       else {
       else {
@@ -268,17 +262,17 @@ public class AppSchedulable extends Schedulable {
 
 
       // If we had previously made a reservation, delete it
       // If we had previously made a reservation, delete it
       if (reserved) {
       if (reserved) {
-        unreserve(priority, node);
+        unreserve(application, priority, node);
       }
       }
 
 
       // Inform the node
       // Inform the node
-      node.allocateContainer(app.getApplicationId(),
+      node.allocateContainer(application.getApplicationId(),
           allocatedContainer);
           allocatedContainer);
 
 
       return container.getResource();
       return container.getResource();
     } else {
     } else {
       // The desired container won't fit here, so reserve
       // The desired container won't fit here, so reserve
-      reserve(priority, node, container, reserved);
+      reserve(application, priority, node, container, reserved);
 
 
       return FairScheduler.CONTAINER_RESERVED;
       return FairScheduler.CONTAINER_RESERVED;
     }
     }
@@ -293,7 +287,7 @@ public class AppSchedulable extends Schedulable {
 
 
       // Make sure the application still needs requests at this priority
       // Make sure the application still needs requests at this priority
       if (app.getTotalRequiredResources(priority) == 0) {
       if (app.getTotalRequiredResources(priority) == 0) {
-        unreserve(priority, node);
+        unreserve(app, priority, node);
         return Resources.none();
         return Resources.none();
       }
       }
     } else {
     } else {
@@ -310,8 +304,7 @@ public class AppSchedulable extends Schedulable {
     // (not scheduled) in order to promote better locality.
     // (not scheduled) in order to promote better locality.
     synchronized (app) {
     synchronized (app) {
       for (Priority priority : prioritiesToTry) {
       for (Priority priority : prioritiesToTry) {
-        if (app.getTotalRequiredResources(priority) <= 0 ||
-            !hasContainerForNode(priority, node)) {
+        if (app.getTotalRequiredResources(priority) <= 0) {
           continue;
           continue;
         }
         }
         
         
@@ -328,14 +321,14 @@ public class AppSchedulable extends Schedulable {
         
         
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
             && localRequest != null && localRequest.getNumContainers() != 0) {
             && localRequest != null && localRequest.getNumContainers() != 0) {
-          return assignContainer(node, priority,
+          return assignContainer(node, app, priority,
               localRequest, NodeType.NODE_LOCAL, reserved);
               localRequest, NodeType.NODE_LOCAL, reserved);
         }
         }
 
 
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
             && (allowedLocality.equals(NodeType.RACK_LOCAL) ||
             && (allowedLocality.equals(NodeType.RACK_LOCAL) ||
                 allowedLocality.equals(NodeType.OFF_SWITCH))) {
                 allowedLocality.equals(NodeType.OFF_SWITCH))) {
-          return assignContainer(node, priority, rackLocalRequest,
+          return assignContainer(node, app, priority, rackLocalRequest,
               NodeType.RACK_LOCAL, reserved);
               NodeType.RACK_LOCAL, reserved);
         }
         }
 
 
@@ -343,7 +336,7 @@ public class AppSchedulable extends Schedulable {
             ResourceRequest.ANY);
             ResourceRequest.ANY);
         if (offSwitchRequest != null && offSwitchRequest.getNumContainers() != 0
         if (offSwitchRequest != null && offSwitchRequest.getNumContainers() != 0
             && allowedLocality.equals(NodeType.OFF_SWITCH)) {
             && allowedLocality.equals(NodeType.OFF_SWITCH)) {
-          return assignContainer(node, priority, offSwitchRequest,
+          return assignContainer(node, app, priority, offSwitchRequest,
               NodeType.OFF_SWITCH, reserved);
               NodeType.OFF_SWITCH, reserved);
         }
         }
       }
       }
@@ -359,16 +352,4 @@ public class AppSchedulable extends Schedulable {
   public Resource assignContainer(FSSchedulerNode node) {
   public Resource assignContainer(FSSchedulerNode node) {
     return assignContainer(node, false);
     return assignContainer(node, false);
   }
   }
-  
-  /**
-   * Whether this app has containers requests that could be satisfied on the
-   * given node, if the node had full space.
-   */
-  public boolean hasContainerForNode(Priority prio, FSSchedulerNode node) {
-    // TODO: add checks stuff about node specific scheduling here
-    ResourceRequest request = app.getResourceRequest(prio, ResourceRequest.ANY);
-    return request.getNumContainers() > 0 && 
-        Resources.lessThanOrEqual(RESOURCE_CALCULATOR, null,
-            request.getCapability(), node.getRMNode().getTotalCapability());
-  }
 }
 }

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 
 public class FSLeafQueue extends FSQueue {
 public class FSLeafQueue extends FSQueue {
   private static final Log LOG = LogFactory.getLog(
   private static final Log LOG = LogFactory.getLog(
@@ -127,8 +126,8 @@ public class FSLeafQueue extends FSQueue {
             + demand);
             + demand);
       }
       }
       demand = Resources.add(demand, toAdd);
       demand = Resources.add(demand, toAdd);
-      demand = Resources.componentwiseMin(demand, maxRes);
-      if (Resources.equals(demand, maxRes)) {
+      if (Resources.greaterThanOrEqual(demand, maxRes)) {
+        demand = maxRes;
         break;
         break;
       }
       }
     }
     }
@@ -154,7 +153,7 @@ public class FSLeafQueue extends FSQueue {
     for (AppSchedulable sched : appScheds) {
     for (AppSchedulable sched : appScheds) {
       if (sched.getRunnable()) {
       if (sched.getRunnable()) {
         assigned = sched.assignContainer(node);
         assigned = sched.assignContainer(node);
-        if (!assigned.equals(Resources.none())) {
+        if (Resources.greaterThan(assigned, Resources.none())) {
           break;
           break;
         }
         }
       }
       }

+ 6 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java

@@ -29,7 +29,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 
 public class FSParentQueue extends FSQueue {
 public class FSParentQueue extends FSQueue {
   private static final Log LOG = LogFactory.getLog(
   private static final Log LOG = LogFactory.getLog(
@@ -88,8 +87,8 @@ public class FSParentQueue extends FSQueue {
             " now " + demand);
             " now " + demand);
       }
       }
       demand = Resources.add(demand, toAdd);
       demand = Resources.add(demand, toAdd);
-      demand = Resources.componentwiseMin(demand, maxRes);
-      if (Resources.equals(demand, maxRes)) {
+      if (Resources.greaterThanOrEqual(demand, maxRes)) {
+        demand = maxRes;
         break;
         break;
       }
       }
     }
     }
@@ -136,14 +135,16 @@ public class FSParentQueue extends FSQueue {
     Resource assigned = Resources.none();
     Resource assigned = Resources.none();
 
 
     // If this queue is over its limit, reject
     // If this queue is over its limit, reject
-    if (!assignContainerPreCheck(node)) {
+    if (Resources.greaterThan(getResourceUsage(),
+        queueMgr.getMaxResources(getName()))) {
       return assigned;
       return assigned;
     }
     }
 
 
     Collections.sort(childQueues, policy.getComparator());
     Collections.sort(childQueues, policy.getComparator());
     for (FSQueue child : childQueues) {
     for (FSQueue child : childQueues) {
       assigned = child.assignContainer(node);
       assigned = child.assignContainer(node);
-      if (!Resources.equals(assigned, Resources.none())) {
+      if (node.getReservedContainer() != null
+          || Resources.greaterThan(assigned, Resources.none())) {
         break;
         break;
       }
       }
     }
     }

+ 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/FSQueue.java

@@ -32,7 +32,6 @@ import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 
 
@@ -162,7 +161,7 @@ public abstract class FSQueue extends Schedulable implements Queue {
    * @return true if check passes (can assign) or false otherwise
    * @return true if check passes (can assign) or false otherwise
    */
    */
   protected boolean assignContainerPreCheck(FSSchedulerNode node) {
   protected boolean assignContainerPreCheck(FSSchedulerNode node) {
-    if (!Resources.fitsIn(getResourceUsage(),
+    if (Resources.greaterThan(getResourceUsage(),
         queueMgr.getMaxResources(getName()))
         queueMgr.getMaxResources(getName()))
         || node.getReservedContainer() != null) {
         || node.getReservedContainer() != null) {
       return false;
       return false;

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

@@ -46,7 +46,7 @@ public class FSSchedulerNode extends SchedulerNode {
   private static final RecordFactory recordFactory = RecordFactoryProvider
   private static final RecordFactory recordFactory = RecordFactoryProvider
       .getRecordFactory(null);
       .getRecordFactory(null);
 
 
-  private Resource availableResource;
+  private Resource availableResource = recordFactory.newRecordInstance(Resource.class);
   private Resource usedResource = recordFactory.newRecordInstance(Resource.class);
   private Resource usedResource = recordFactory.newRecordInstance(Resource.class);
 
 
   private volatile int numContainers;
   private volatile int numContainers;
@@ -62,7 +62,7 @@ public class FSSchedulerNode extends SchedulerNode {
 
 
   public FSSchedulerNode(RMNode node) {
   public FSSchedulerNode(RMNode node) {
     this.rmNode = node;
     this.rmNode = node;
-    this.availableResource = Resources.clone(node.getTotalCapability());
+    this.availableResource.setMemory(node.getTotalCapability().getMemory());
   }
   }
 
 
   public RMNode getRMNode() {
   public RMNode getRMNode() {

+ 27 - 56
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -41,7 +41,6 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
@@ -53,9 +52,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@@ -115,9 +111,6 @@ public class FairScheduler implements ResourceScheduler {
 
 
   private static final Log LOG = LogFactory.getLog(FairScheduler.class);
   private static final Log LOG = LogFactory.getLog(FairScheduler.class);
   
   
-  private static final ResourceCalculator RESOURCE_CALCULATOR =
-      new DefaultResourceCalculator();
-  
   // Value that container assignment methods return when a container is
   // Value that container assignment methods return when a container is
   // reserved
   // reserved
   public static final Resource CONTAINER_RESERVED = Resources.createResource(-1);
   public static final Resource CONTAINER_RESERVED = Resources.createResource(-1);
@@ -253,10 +246,8 @@ public class FairScheduler implements ResourceScheduler {
    * Is a queue below its min share for the given task type?
    * Is a queue below its min share for the given task type?
    */
    */
   boolean isStarvedForMinShare(FSLeafQueue sched) {
   boolean isStarvedForMinShare(FSLeafQueue sched) {
-    Resource desiredShare = Resources.min(RESOURCE_CALCULATOR, clusterCapacity,
-        sched.getMinShare(), sched.getDemand());
-    return Resources.lessThan(RESOURCE_CALCULATOR, clusterCapacity,
-        sched.getResourceUsage(), desiredShare);
+    Resource desiredShare = Resources.min(sched.getMinShare(), sched.getDemand());
+    return Resources.lessThan(sched.getResourceUsage(), desiredShare);
   }
   }
 
 
   /**
   /**
@@ -264,10 +255,9 @@ public class FairScheduler implements ResourceScheduler {
    * defined as being below half its fair share.
    * defined as being below half its fair share.
    */
    */
   boolean isStarvedForFairShare(FSLeafQueue sched) {
   boolean isStarvedForFairShare(FSLeafQueue sched) {
-    Resource desiredFairShare = Resources.max(RESOURCE_CALCULATOR, clusterCapacity,
+    Resource desiredFairShare = Resources.max(
         Resources.multiply(sched.getFairShare(), .5), sched.getDemand());
         Resources.multiply(sched.getFairShare(), .5), sched.getDemand());
-    return Resources.lessThan(RESOURCE_CALCULATOR, clusterCapacity,
-        sched.getResourceUsage(), desiredFairShare);
+    return Resources.lessThan(sched.getResourceUsage(), desiredFairShare);
   }
   }
 
 
   /**
   /**
@@ -293,8 +283,7 @@ public class FairScheduler implements ResourceScheduler {
     for (FSLeafQueue sched : queueMgr.getLeafQueues()) {
     for (FSLeafQueue sched : queueMgr.getLeafQueues()) {
       resToPreempt = Resources.add(resToPreempt, resToPreempt(sched, curTime));
       resToPreempt = Resources.add(resToPreempt, resToPreempt(sched, curTime));
     }
     }
-    if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity, resToPreempt,
-        Resources.none())) {
+    if (Resources.greaterThan(resToPreempt, Resources.none())) {
       preemptResources(queueMgr.getLeafQueues(), resToPreempt);
       preemptResources(queueMgr.getLeafQueues(), resToPreempt);
     }
     }
   }
   }
@@ -320,8 +309,7 @@ public class FairScheduler implements ResourceScheduler {
     // Collect running containers from over-scheduled queues
     // Collect running containers from over-scheduled queues
     List<RMContainer> runningContainers = new ArrayList<RMContainer>();
     List<RMContainer> runningContainers = new ArrayList<RMContainer>();
     for (FSLeafQueue sched : scheds) {
     for (FSLeafQueue sched : scheds) {
-      if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
-          sched.getResourceUsage(), sched.getFairShare())) {
+      if (Resources.greaterThan(sched.getResourceUsage(), sched.getFairShare())) {
         for (AppSchedulable as : sched.getAppSchedulables()) {
         for (AppSchedulable as : sched.getAppSchedulables()) {
           for (RMContainer c : as.getApp().getLiveContainers()) {
           for (RMContainer c : as.getApp().getLiveContainers()) {
             runningContainers.add(c);
             runningContainers.add(c);
@@ -344,8 +332,7 @@ public class FairScheduler implements ResourceScheduler {
     // tasks, making sure we don't kill too many from any queue
     // tasks, making sure we don't kill too many from any queue
     for (RMContainer container : runningContainers) {
     for (RMContainer container : runningContainers) {
       FSLeafQueue sched = queues.get(container);
       FSLeafQueue sched = queues.get(container);
-      if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
-          sched.getResourceUsage(), sched.getFairShare())) {
+      if (Resources.greaterThan(sched.getResourceUsage(), sched.getFairShare())) {
         LOG.info("Preempting container (prio=" + container.getContainer().getPriority() +
         LOG.info("Preempting container (prio=" + container.getContainer().getPriority() +
             "res=" + container.getContainer().getResource() +
             "res=" + container.getContainer().getResource() +
             ") from queue " + sched.getName());
             ") from queue " + sched.getName());
@@ -358,8 +345,7 @@ public class FairScheduler implements ResourceScheduler {
 
 
         toPreempt = Resources.subtract(toPreempt,
         toPreempt = Resources.subtract(toPreempt,
             container.getContainer().getResource());
             container.getContainer().getResource());
-        if (Resources.lessThanOrEqual(RESOURCE_CALCULATOR, clusterCapacity,
-            toPreempt, Resources.none())) {
+        if (Resources.equals(toPreempt, Resources.none())) {
           break;
           break;
         }
         }
       }
       }
@@ -383,21 +369,17 @@ public class FairScheduler implements ResourceScheduler {
     Resource resDueToMinShare = Resources.none();
     Resource resDueToMinShare = Resources.none();
     Resource resDueToFairShare = Resources.none();
     Resource resDueToFairShare = Resources.none();
     if (curTime - sched.getLastTimeAtMinShare() > minShareTimeout) {
     if (curTime - sched.getLastTimeAtMinShare() > minShareTimeout) {
-      Resource target = Resources.min(RESOURCE_CALCULATOR, clusterCapacity,
-          sched.getMinShare(), sched.getDemand());
-      resDueToMinShare = Resources.max(RESOURCE_CALCULATOR, clusterCapacity,
-          Resources.none(), Resources.subtract(target, sched.getResourceUsage()));
+      Resource target = Resources.min(sched.getMinShare(), sched.getDemand());
+      resDueToMinShare = Resources.max(Resources.none(),
+          Resources.subtract(target, sched.getResourceUsage()));
     }
     }
     if (curTime - sched.getLastTimeAtHalfFairShare() > fairShareTimeout) {
     if (curTime - sched.getLastTimeAtHalfFairShare() > fairShareTimeout) {
-      Resource target = Resources.min(RESOURCE_CALCULATOR, clusterCapacity,
-          sched.getFairShare(), sched.getDemand());
-      resDueToFairShare = Resources.max(RESOURCE_CALCULATOR, clusterCapacity,
-          Resources.none(), Resources.subtract(target, sched.getResourceUsage()));
-    }
-    Resource resToPreempt = Resources.max(RESOURCE_CALCULATOR, clusterCapacity,
-        resDueToMinShare, resDueToFairShare);
-    if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
-        resToPreempt, Resources.none())) {
+      Resource target = Resources.min(sched.getFairShare(), sched.getDemand());
+      resDueToFairShare = Resources.max(Resources.none(),
+          Resources.subtract(target, sched.getResourceUsage()));
+    }
+    Resource resToPreempt = Resources.max(resDueToMinShare, resDueToFairShare);
+    if (Resources.greaterThan(resToPreempt, Resources.none())) {
       String message = "Should preempt " + resToPreempt + " res for queue "
       String message = "Should preempt " + resToPreempt + " res for queue "
           + sched.getName() + ": resDueToMinShare = " + resDueToMinShare
           + sched.getName() + ": resDueToMinShare = " + resDueToMinShare
           + ", resDueToFairShare = " + resDueToFairShare;
           + ", resDueToFairShare = " + resDueToFairShare;
@@ -806,32 +788,21 @@ public class FairScheduler implements ResourceScheduler {
 
 
     AppSchedulable reservedAppSchedulable = node.getReservedAppSchedulable();
     AppSchedulable reservedAppSchedulable = node.getReservedAppSchedulable();
     if (reservedAppSchedulable != null) {
     if (reservedAppSchedulable != null) {
-      Priority reservedPriority = node.getReservedContainer().getReservedPriority();
-      if (reservedAppSchedulable != null &&
-          !reservedAppSchedulable.hasContainerForNode(reservedPriority, node)) {
-        // Don't hold the reservation if app can no longer use it
-        LOG.info("Releasing reservation that cannot be satisfied for application "
-            + reservedAppSchedulable.getApp().getApplicationAttemptId()
-            + " on node " + nm);
-        reservedAppSchedulable.unreserve(reservedPriority, node);
-        reservedAppSchedulable = null;
-      } else {
-        // Reservation exists; try to fulfill the reservation
-        LOG.info("Trying to fulfill reservation for application "
-            + reservedAppSchedulable.getApp().getApplicationAttemptId()
-            + " on node: " + nm);
-
-        node.getReservedAppSchedulable().assignReservedContainer(node);
-      }
+      // Reservation exists; try to fulfill the reservation
+      LOG.info("Trying to fulfill reservation for application "
+          + reservedAppSchedulable.getApp().getApplicationAttemptId()
+          + " on node: " + nm);
+
+      node.getReservedAppSchedulable().assignReservedContainer(node);
     }
     }
-    if (reservedAppSchedulable == null) {
+    else {
       // No reservation, schedule at queue which is farthest below fair share
       // No reservation, schedule at queue which is farthest below fair share
       int assignedContainers = 0;
       int assignedContainers = 0;
       while (node.getReservedContainer() == null) {
       while (node.getReservedContainer() == null) {
         boolean assignedContainer = false;
         boolean assignedContainer = false;
-        if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
-              queueMgr.getRootQueue().assignContainer(node),
-              Resources.none())) {
+        if (Resources.greaterThan(
+            queueMgr.getRootQueue().assignContainer(node),
+            Resources.none())) {
           assignedContainer = true;
           assignedContainer = true;
         }
         }
         if (!assignedContainer) { break; }
         if (!assignedContainer) { break; }

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.w3c.dom.Document;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 import org.w3c.dom.Node;
@@ -475,8 +474,8 @@ public class QueueManager {
     }
     }
     queueAcls.put(queueName, acls);
     queueAcls.put(queueName, acls);
     if (maxQueueResources.containsKey(queueName) && minQueueResources.containsKey(queueName)
     if (maxQueueResources.containsKey(queueName) && minQueueResources.containsKey(queueName)
-        && !Resources.fitsIn(minQueueResources.get(queueName),
-            maxQueueResources.get(queueName))) {
+        && Resources.lessThan(maxQueueResources.get(queueName),
+            minQueueResources.get(queueName))) {
       LOG.warn(String.format("Queue %s has max resources %d less than min resources %d",
       LOG.warn(String.format("Queue %s has max resources %d less than min resources %d",
           queueName, maxQueueResources.get(queueName), minQueueResources.get(queueName)));
           queueName, maxQueueResources.get(queueName), minQueueResources.get(queueName)));
     }
     }
@@ -505,7 +504,7 @@ public class QueueManager {
     if (maxQueueResource != null) {
     if (maxQueueResource != null) {
       return maxQueueResource;
       return maxQueueResource;
     } else {
     } else {
-      return Resources.createResource(Integer.MAX_VALUE, Integer.MAX_VALUE);
+      return Resources.createResource(Integer.MAX_VALUE);
     }
     }
   }
   }
 
 

+ 150 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Resources.java

@@ -0,0 +1,150 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.Records;
+
+@Private
+@Evolving
+public class Resources {
+  
+  // Java doesn't have const :(
+  private static final Resource NONE = new Resource() {
+
+    @Override
+    public int getMemory() {
+      return 0;
+    }
+
+    @Override
+    public void setMemory(int memory) {
+      throw new RuntimeException("NONE cannot be modified!");
+    }
+
+    @Override
+    public int getVirtualCores() {
+      return 0;
+    }
+
+    @Override
+    public void setVirtualCores(int cores) {
+      throw new RuntimeException("NONE cannot be modified!");
+    }
+
+    @Override
+    public int compareTo(Resource o) {
+      int diff = 0 - o.getMemory();
+      if (diff == 0) {
+        diff = 0 - o.getVirtualCores();
+      }
+      return diff;
+    }
+    
+  };
+
+  public static Resource createResource(int memory) {
+    return createResource(memory, (memory > 0) ? 1 : 0);
+  }
+
+  public static Resource createResource(int memory, int cores) {
+    Resource resource = Records.newRecord(Resource.class);
+    resource.setMemory(memory);
+    resource.setVirtualCores(cores);
+    return resource;
+  }
+
+  public static Resource none() {
+    return NONE;
+  }
+
+  public static Resource clone(Resource res) {
+    return createResource(res.getMemory(), res.getVirtualCores());
+  }
+
+  public static Resource addTo(Resource lhs, Resource rhs) {
+    lhs.setMemory(lhs.getMemory() + rhs.getMemory());
+    return lhs;
+  }
+
+  public static Resource add(Resource lhs, Resource rhs) {
+    return addTo(clone(lhs), rhs);
+  }
+
+  public static Resource subtractFrom(Resource lhs, Resource rhs) {
+    lhs.setMemory(lhs.getMemory() - rhs.getMemory());
+    return lhs;
+  }
+
+  public static Resource subtract(Resource lhs, Resource rhs) {
+    return subtractFrom(clone(lhs), rhs);
+  }
+
+  public static Resource negate(Resource resource) {
+    return subtract(NONE, resource);
+  }
+
+  public static Resource multiplyTo(Resource lhs, int by) {
+    lhs.setMemory(lhs.getMemory() * by);
+    return lhs;
+  }
+
+  public static Resource multiply(Resource lhs, int by) {
+    return multiplyTo(clone(lhs), by);
+  }
+  
+  /**
+   * Mutliply a resource by a {@code double}. Note that integral 
+   * resource quantites are subject to rounding during cast.
+   */
+  public static Resource multiply(Resource lhs, double by) {
+    Resource out = clone(lhs);
+    out.setMemory((int) (lhs.getMemory() * by));
+    return out;
+  }
+
+  public static boolean equals(Resource lhs, Resource rhs) {
+    return lhs.getMemory() == rhs.getMemory();
+  }
+
+  public static boolean lessThan(Resource lhs, Resource rhs) {
+    return lhs.getMemory() < rhs.getMemory();
+  }
+
+  public static boolean lessThanOrEqual(Resource lhs, Resource rhs) {
+    return lhs.getMemory() <= rhs.getMemory();
+  }
+
+  public static boolean greaterThan(Resource lhs, Resource rhs) {
+    return lhs.getMemory() > rhs.getMemory();
+  }
+
+  public static boolean greaterThanOrEqual(Resource lhs, Resource rhs) {
+    return lhs.getMemory() >= rhs.getMemory();
+  }
+  
+  public static Resource min(Resource lhs, Resource rhs) {
+    return (lhs.getMemory() < rhs.getMemory()) ? lhs : rhs;
+  }
+
+  public static Resource max(Resource lhs, Resource rhs) {
+    return (lhs.getMemory() > rhs.getMemory()) ? lhs : rhs;
+  }}

+ 10 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java

@@ -22,8 +22,7 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.Comparator;
 
 
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
 
 
@@ -32,8 +31,6 @@ import com.google.common.annotations.VisibleForTesting;
 public class FairSharePolicy extends SchedulingPolicy {
 public class FairSharePolicy extends SchedulingPolicy {
   @VisibleForTesting
   @VisibleForTesting
   public static final String NAME = "Fairshare";
   public static final String NAME = "Fairshare";
-  private static final DefaultResourceCalculator RESOURCE_CALCULATOR =
-      new DefaultResourceCalculator();
   private FairShareComparator comparator = new FairShareComparator();
   private FairShareComparator comparator = new FairShareComparator();
 
 
   @Override
   @Override
@@ -62,19 +59,15 @@ public class FairSharePolicy extends SchedulingPolicy {
     public int compare(Schedulable s1, Schedulable s2) {
     public int compare(Schedulable s1, Schedulable s2) {
       double minShareRatio1, minShareRatio2;
       double minShareRatio1, minShareRatio2;
       double useToWeightRatio1, useToWeightRatio2;
       double useToWeightRatio1, useToWeightRatio2;
-      Resource minShare1 = Resources.min(RESOURCE_CALCULATOR, null,
-          s1.getMinShare(), s1.getDemand());
-      Resource minShare2 = Resources.min(RESOURCE_CALCULATOR, null,
-          s2.getMinShare(), s2.getDemand());
-      boolean s1Needy = Resources.lessThan(RESOURCE_CALCULATOR, null,
-          s1.getResourceUsage(), minShare1);
-      boolean s2Needy = Resources.lessThan(RESOURCE_CALCULATOR, null,
-          s2.getResourceUsage(), minShare2);
+      Resource minShare1 = Resources.min(s1.getMinShare(), s1.getDemand());
+      Resource minShare2 = Resources.min(s2.getMinShare(), s2.getDemand());
+      boolean s1Needy = Resources.lessThan(s1.getResourceUsage(), minShare1);
+      boolean s2Needy = Resources.lessThan(s2.getResourceUsage(), minShare2);
       Resource one = Resources.createResource(1);
       Resource one = Resources.createResource(1);
       minShareRatio1 = (double) s1.getResourceUsage().getMemory()
       minShareRatio1 = (double) s1.getResourceUsage().getMemory()
-          / Resources.max(RESOURCE_CALCULATOR, null, minShare1, one).getMemory();
+          / Resources.max(minShare1, one).getMemory();
       minShareRatio2 = (double) s2.getResourceUsage().getMemory()
       minShareRatio2 = (double) s2.getResourceUsage().getMemory()
-          / Resources.max(RESOURCE_CALCULATOR, null, minShare2, one).getMemory();
+          / Resources.max(minShare2, one).getMemory();
       useToWeightRatio1 = s1.getResourceUsage().getMemory() / s1.getWeight();
       useToWeightRatio1 = s1.getResourceUsage().getMemory() / s1.getWeight();
       useToWeightRatio2 = s2.getResourceUsage().getMemory() / s2.getWeight();
       useToWeightRatio2 = s2.getResourceUsage().getMemory() / s2.getWeight();
       int res = 0;
       int res = 0;
@@ -168,11 +161,9 @@ public class FairSharePolicy extends SchedulingPolicy {
     for (Schedulable sched : schedulables) {
     for (Schedulable sched : schedulables) {
       Resources.addTo(totalDemand, sched.getDemand());
       Resources.addTo(totalDemand, sched.getDemand());
     }
     }
-    Resource cap = Resources.min(RESOURCE_CALCULATOR, null, totalDemand,
-        totalResources);
+    Resource cap = Resources.min(totalDemand, totalResources);
     double rMax = 1.0;
     double rMax = 1.0;
-    while (Resources.lessThan(RESOURCE_CALCULATOR, null,
-        resUsedWithWeightToResRatio(rMax, schedulables),
+    while (Resources.lessThan(resUsedWithWeightToResRatio(rMax, schedulables),
         cap)) {
         cap)) {
       rMax *= 2.0;
       rMax *= 2.0;
     }
     }
@@ -181,8 +172,7 @@ public class FairSharePolicy extends SchedulingPolicy {
     double right = rMax;
     double right = rMax;
     for (int i = 0; i < COMPUTE_FAIR_SHARES_ITERATIONS; i++) {
     for (int i = 0; i < COMPUTE_FAIR_SHARES_ITERATIONS; i++) {
       double mid = (left + right) / 2.0;
       double mid = (left + right) / 2.0;
-      if (Resources.lessThan(RESOURCE_CALCULATOR, null,
-          resUsedWithWeightToResRatio(mid, schedulables),
+      if (Resources.lessThan(resUsedWithWeightToResRatio(mid, schedulables),
           cap)) {
           cap)) {
         left = mid;
         left = mid;
       } else {
       } else {

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

@@ -22,8 +22,8 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.Comparator;
 
 
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;

+ 29 - 58
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedAction;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
@@ -29,10 +28,11 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.io.DataInputByteBuffer;
-import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ContainerManager;
 import org.apache.hadoop.yarn.api.ContainerManager;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
@@ -59,17 +59,9 @@ public class TestAMAuthorization {
 
 
   private static final Log LOG = LogFactory.getLog(TestAMAuthorization.class);
   private static final Log LOG = LogFactory.getLog(TestAMAuthorization.class);
 
 
-  private static final Configuration confWithSecurityEnabled =
-      new Configuration();
-  static {
-    confWithSecurityEnabled.set(
-      CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(confWithSecurityEnabled);
-  }
-
   public static final class MyContainerManager implements ContainerManager {
   public static final class MyContainerManager implements ContainerManager {
 
 
-    public ByteBuffer amTokens;
+    public Map<String, String> amContainerEnv;
 
 
     public MyContainerManager() {
     public MyContainerManager() {
     }
     }
@@ -78,7 +70,7 @@ public class TestAMAuthorization {
     public StartContainerResponse
     public StartContainerResponse
         startContainer(StartContainerRequest request)
         startContainer(StartContainerRequest request)
             throws YarnRemoteException {
             throws YarnRemoteException {
-      amTokens = request.getContainerLaunchContext().getContainerTokens();
+      amContainerEnv = request.getContainerLaunchContext().getEnvironment();
       return null;
       return null;
     }
     }
 
 
@@ -101,6 +93,9 @@ public class TestAMAuthorization {
 
 
     public MockRMWithAMS(Configuration conf, ContainerManager containerManager) {
     public MockRMWithAMS(Configuration conf, ContainerManager containerManager) {
       super(conf, containerManager);
       super(conf, containerManager);
+      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+          "kerberos");
+      UserGroupInformation.setConfiguration(conf);
     }
     }
 
 
     @Override
     @Override
@@ -110,6 +105,7 @@ public class TestAMAuthorization {
 
 
     @Override
     @Override
     protected ApplicationMasterService createApplicationMasterService() {
     protected ApplicationMasterService createApplicationMasterService() {
+
       return new ApplicationMasterService(getRMContext(), this.scheduler);
       return new ApplicationMasterService(getRMContext(), this.scheduler);
     }
     }
   }
   }
@@ -117,8 +113,7 @@ public class TestAMAuthorization {
   @Test
   @Test
   public void testAuthorizedAccess() throws Exception {
   public void testAuthorizedAccess() throws Exception {
     MyContainerManager containerManager = new MyContainerManager();
     MyContainerManager containerManager = new MyContainerManager();
-    final MockRM rm =
-        new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+    final MockRM rm = new MockRMWithAMS(new Configuration(), containerManager);
     rm.start();
     rm.start();
 
 
     MockNM nm1 = rm.registerNode("localhost:1234", 5120);
     MockNM nm1 = rm.registerNode("localhost:1234", 5120);
@@ -131,11 +126,11 @@ public class TestAMAuthorization {
     nm1.nodeHeartbeat(true);
     nm1.nodeHeartbeat(true);
 
 
     int waitCount = 0;
     int waitCount = 0;
-    while (containerManager.amTokens == null && waitCount++ < 20) {
+    while (containerManager.amContainerEnv == null && waitCount++ < 20) {
       LOG.info("Waiting for AM Launch to happen..");
       LOG.info("Waiting for AM Launch to happen..");
       Thread.sleep(1000);
       Thread.sleep(1000);
     }
     }
-    Assert.assertNotNull(containerManager.amTokens);
+    Assert.assertNotNull(containerManager.amContainerEnv);
 
 
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
     ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -147,12 +142,12 @@ public class TestAMAuthorization {
 
 
     UserGroupInformation currentUser = UserGroupInformation
     UserGroupInformation currentUser = UserGroupInformation
         .createRemoteUser(applicationAttemptId.toString());
         .createRemoteUser(applicationAttemptId.toString());
-    Credentials credentials = new Credentials();
-    DataInputByteBuffer buf = new DataInputByteBuffer();
-    containerManager.amTokens.rewind();
-    buf.reset(containerManager.amTokens);
-    credentials.readTokenStorageStream(buf);
-    currentUser.addCredentials(credentials);
+    String tokenURLEncodedStr = containerManager.amContainerEnv
+        .get(ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME);
+    LOG.info("AppMasterToken is " + tokenURLEncodedStr);
+    Token<? extends TokenIdentifier> token = new Token<TokenIdentifier>();
+    token.decodeFromUrlString(tokenURLEncodedStr);
+    currentUser.addToken(token);
 
 
     AMRMProtocol client = currentUser
     AMRMProtocol client = currentUser
         .doAs(new PrivilegedAction<AMRMProtocol>() {
         .doAs(new PrivilegedAction<AMRMProtocol>() {
@@ -177,7 +172,7 @@ public class TestAMAuthorization {
   @Test
   @Test
   public void testUnauthorizedAccess() throws Exception {
   public void testUnauthorizedAccess() throws Exception {
     MyContainerManager containerManager = new MyContainerManager();
     MyContainerManager containerManager = new MyContainerManager();
-    MockRM rm = new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+    MockRM rm = new MockRMWithAMS(new Configuration(), containerManager);
     rm.start();
     rm.start();
 
 
     MockNM nm1 = rm.registerNode("localhost:1234", 5120);
     MockNM nm1 = rm.registerNode("localhost:1234", 5120);
@@ -187,16 +182,17 @@ public class TestAMAuthorization {
     nm1.nodeHeartbeat(true);
     nm1.nodeHeartbeat(true);
 
 
     int waitCount = 0;
     int waitCount = 0;
-    while (containerManager.amTokens == null && waitCount++ < 20) {
+    while (containerManager.amContainerEnv == null && waitCount++ < 20) {
       LOG.info("Waiting for AM Launch to happen..");
       LOG.info("Waiting for AM Launch to happen..");
       Thread.sleep(1000);
       Thread.sleep(1000);
     }
     }
-    Assert.assertNotNull(containerManager.amTokens);
+    Assert.assertNotNull(containerManager.amContainerEnv);
 
 
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
     ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
     waitForLaunchedState(attempt);
     waitForLaunchedState(attempt);
 
 
+    // Create a client to the RM.
     final Configuration conf = rm.getConfig();
     final Configuration conf = rm.getConfig();
     final YarnRPC rpc = YarnRPC.create(conf);
     final YarnRPC rpc = YarnRPC.create(conf);
     final InetSocketAddress serviceAddr = conf.getSocketAddr(
     final InetSocketAddress serviceAddr = conf.getSocketAddr(
@@ -206,8 +202,13 @@ public class TestAMAuthorization {
 
 
     UserGroupInformation currentUser = UserGroupInformation
     UserGroupInformation currentUser = UserGroupInformation
         .createRemoteUser(applicationAttemptId.toString());
         .createRemoteUser(applicationAttemptId.toString());
+    String tokenURLEncodedStr = containerManager.amContainerEnv
+        .get(ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME);
+    LOG.info("AppMasterToken is " + tokenURLEncodedStr);
+    Token<? extends TokenIdentifier> token = new Token<TokenIdentifier>();
+    token.decodeFromUrlString(tokenURLEncodedStr);
+    currentUser.addToken(token);
 
 
-    // First try contacting NM without tokens
     AMRMProtocol client = currentUser
     AMRMProtocol client = currentUser
         .doAs(new PrivilegedAction<AMRMProtocol>() {
         .doAs(new PrivilegedAction<AMRMProtocol>() {
           @Override
           @Override
@@ -216,39 +217,9 @@ public class TestAMAuthorization {
                 serviceAddr, conf);
                 serviceAddr, conf);
           }
           }
         });
         });
+
     RegisterApplicationMasterRequest request = Records
     RegisterApplicationMasterRequest request = Records
         .newRecord(RegisterApplicationMasterRequest.class);
         .newRecord(RegisterApplicationMasterRequest.class);
-    request.setApplicationAttemptId(applicationAttemptId);
-    try {
-      client.registerApplicationMaster(request);
-      Assert.fail("Should fail with authorization error");
-    } catch (Exception e) {
-      // Because there are no tokens, the request should be rejected as the
-      // server side will assume we are trying simple auth.
-      Assert.assertTrue(e.getCause().getMessage().contains(
-        "SIMPLE authentication is not enabled.  "
-            + "Available:[KERBEROS, DIGEST]"));
-    }
-
-    // Now try to validate invalid authorization.
-    Credentials credentials = new Credentials();
-    DataInputByteBuffer buf = new DataInputByteBuffer();
-    containerManager.amTokens.rewind();
-    buf.reset(containerManager.amTokens);
-    credentials.readTokenStorageStream(buf);
-    currentUser.addCredentials(credentials);
-
-    // Create a client to the RM.
-    client = currentUser
-        .doAs(new PrivilegedAction<AMRMProtocol>() {
-          @Override
-          public AMRMProtocol run() {
-            return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class,
-                serviceAddr, conf);
-          }
-        });
-
-    request = Records.newRecord(RegisterApplicationMasterRequest.class);
     ApplicationAttemptId otherAppAttemptId = BuilderUtils
     ApplicationAttemptId otherAppAttemptId = BuilderUtils
         .newApplicationAttemptId(applicationAttemptId.getApplicationId(), 42);
         .newApplicationAttemptId(applicationAttemptId.getApplicationId(), 42);
     request.setApplicationAttemptId(otherAppAttemptId);
     request.setApplicationAttemptId(otherAppAttemptId);

+ 0 - 43
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResources.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.server.resourcemanager.resource;
-
-import static org.apache.hadoop.yarn.server.resourcemanager.resource.Resources.*;
-import static org.junit.Assert.*;
-import org.junit.Test;
-
-public class TestResources {
-  @Test(timeout=1000)
-  public void testFitsIn() {
-    assertTrue(fitsIn(createResource(1, 1), createResource(2, 2)));
-    assertTrue(fitsIn(createResource(2, 2), createResource(2, 2)));
-    assertFalse(fitsIn(createResource(2, 2), createResource(1, 1)));
-    assertFalse(fitsIn(createResource(1, 2), createResource(2, 1)));
-    assertFalse(fitsIn(createResource(2, 1), createResource(1, 2)));
-  }
-  
-  @Test(timeout=1000)
-  public void testComponentwiseMin() {
-    assertEquals(createResource(1, 1),
-        componentwiseMin(createResource(1, 1), createResource(2, 2)));
-    assertEquals(createResource(1, 1),
-        componentwiseMin(createResource(2, 2), createResource(1, 1)));
-    assertEquals(createResource(1, 1),
-        componentwiseMin(createResource(1, 2), createResource(2, 1)));
-  }
-}

+ 0 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -1520,28 +1520,4 @@ public class TestFairScheduler {
     }
     }
     assertEquals(FinalApplicationStatus.FAILED, application.getFinalApplicationStatus());
     assertEquals(FinalApplicationStatus.FAILED, application.getFinalApplicationStatus());
   }
   }
-  
-  @Test
-  public void testReservationThatDoesntFit() {
-    RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
-    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
-    scheduler.handle(nodeEvent1);
-    
-    ApplicationAttemptId attId = createSchedulingRequest(2048, "queue1",
-        "user1", 1);
-    scheduler.update();
-    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
-    scheduler.handle(updateEvent);
-    
-    FSSchedulerApp app = scheduler.applications.get(attId);
-    assertEquals(0, app.getLiveContainers().size());
-    assertEquals(0, app.getReservedContainers().size());
-    
-    createSchedulingRequestExistingApplication(1024, 2, attId);
-    scheduler.update();
-    scheduler.handle(updateEvent);
-    
-    assertEquals(1, app.getLiveContainers().size());
-    assertEquals(0, app.getReservedContainers().size());
-  }
 }
 }

+ 24 - 31
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java

@@ -25,11 +25,11 @@ import javax.crypto.SecretKey;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.io.DataInputByteBuffer;
-import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
@@ -51,14 +51,6 @@ public class TestApplicationTokens {
 
 
   private static final Log LOG = LogFactory.getLog(TestApplicationTokens.class);
   private static final Log LOG = LogFactory.getLog(TestApplicationTokens.class);
 
 
-  private static final Configuration confWithSecurityEnabled =
-      new Configuration();
-  static {
-    confWithSecurityEnabled.set(
-      CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(confWithSecurityEnabled);
-  }
-
   /**
   /**
    * Validate that application tokens are unusable after the
    * Validate that application tokens are unusable after the
    * application-finishes.
    * application-finishes.
@@ -69,8 +61,7 @@ public class TestApplicationTokens {
   public void testTokenExpiry() throws Exception {
   public void testTokenExpiry() throws Exception {
 
 
     MyContainerManager containerManager = new MyContainerManager();
     MyContainerManager containerManager = new MyContainerManager();
-    final MockRM rm =
-        new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+    final MockRM rm = new MockRMWithAMS(new Configuration(), containerManager);
     rm.start();
     rm.start();
 
 
     final Configuration conf = rm.getConfig();
     final Configuration conf = rm.getConfig();
@@ -85,11 +76,11 @@ public class TestApplicationTokens {
       nm1.nodeHeartbeat(true);
       nm1.nodeHeartbeat(true);
 
 
       int waitCount = 0;
       int waitCount = 0;
-      while (containerManager.amTokens == null && waitCount++ < 20) {
+      while (containerManager.amContainerEnv == null && waitCount++ < 20) {
         LOG.info("Waiting for AM Launch to happen..");
         LOG.info("Waiting for AM Launch to happen..");
         Thread.sleep(1000);
         Thread.sleep(1000);
       }
       }
-      Assert.assertNotNull(containerManager.amTokens);
+      Assert.assertNotNull(containerManager.amContainerEnv);
 
 
       RMAppAttempt attempt = app.getCurrentAppAttempt();
       RMAppAttempt attempt = app.getCurrentAppAttempt();
       ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
       ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -98,12 +89,13 @@ public class TestApplicationTokens {
       UserGroupInformation currentUser =
       UserGroupInformation currentUser =
           UserGroupInformation
           UserGroupInformation
             .createRemoteUser(applicationAttemptId.toString());
             .createRemoteUser(applicationAttemptId.toString());
-      Credentials credentials = new Credentials();
-      DataInputByteBuffer buf = new DataInputByteBuffer();
-      containerManager.amTokens.rewind();
-      buf.reset(containerManager.amTokens);
-      credentials.readTokenStorageStream(buf);
-      currentUser.addCredentials(credentials);
+      String tokenURLEncodedStr =
+          containerManager.amContainerEnv
+            .get(ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME);
+      LOG.info("AppMasterToken is " + tokenURLEncodedStr);
+      Token<? extends TokenIdentifier> token = new Token<TokenIdentifier>();
+      token.decodeFromUrlString(tokenURLEncodedStr);
+      currentUser.addToken(token);
 
 
       rmClient = createRMClient(rm, conf, rpc, currentUser);
       rmClient = createRMClient(rm, conf, rpc, currentUser);
 
 
@@ -160,9 +152,9 @@ public class TestApplicationTokens {
   @Test
   @Test
   public void testMasterKeyRollOver() throws Exception {
   public void testMasterKeyRollOver() throws Exception {
 
 
+    Configuration config = new Configuration();
     MyContainerManager containerManager = new MyContainerManager();
     MyContainerManager containerManager = new MyContainerManager();
-    final MockRM rm =
-        new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+    final MockRM rm = new MockRMWithAMS(config, containerManager);
     rm.start();
     rm.start();
 
 
     final Configuration conf = rm.getConfig();
     final Configuration conf = rm.getConfig();
@@ -177,11 +169,11 @@ public class TestApplicationTokens {
       nm1.nodeHeartbeat(true);
       nm1.nodeHeartbeat(true);
 
 
       int waitCount = 0;
       int waitCount = 0;
-      while (containerManager.amTokens == null && waitCount++ < 20) {
+      while (containerManager.amContainerEnv == null && waitCount++ < 20) {
         LOG.info("Waiting for AM Launch to happen..");
         LOG.info("Waiting for AM Launch to happen..");
         Thread.sleep(1000);
         Thread.sleep(1000);
       }
       }
-      Assert.assertNotNull(containerManager.amTokens);
+      Assert.assertNotNull(containerManager.amContainerEnv);
 
 
       RMAppAttempt attempt = app.getCurrentAppAttempt();
       RMAppAttempt attempt = app.getCurrentAppAttempt();
       ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
       ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -190,12 +182,13 @@ public class TestApplicationTokens {
       UserGroupInformation currentUser =
       UserGroupInformation currentUser =
           UserGroupInformation
           UserGroupInformation
             .createRemoteUser(applicationAttemptId.toString());
             .createRemoteUser(applicationAttemptId.toString());
-      Credentials credentials = new Credentials();
-      DataInputByteBuffer buf = new DataInputByteBuffer();
-      containerManager.amTokens.rewind();
-      buf.reset(containerManager.amTokens);
-      credentials.readTokenStorageStream(buf);
-      currentUser.addCredentials(credentials);
+      String tokenURLEncodedStr =
+          containerManager.amContainerEnv
+            .get(ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME);
+      LOG.info("AppMasterToken is " + tokenURLEncodedStr);
+      Token<? extends TokenIdentifier> token = new Token<TokenIdentifier>();
+      token.decodeFromUrlString(tokenURLEncodedStr);
+      currentUser.addToken(token);
 
 
       rmClient = createRMClient(rm, conf, rpc, currentUser);
       rmClient = createRMClient(rm, conf, rpc, currentUser);
 
 

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

@@ -293,14 +293,14 @@ public class TestRMWebServices extends JerseyTest {
 
 
     WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn",
     WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn",
         VersionInfo.getDate(), hadoopVersionBuiltOn);
         VersionInfo.getDate(), hadoopVersionBuiltOn);
-    WebServicesTestUtils.checkStringEqual("hadoopBuildVersion",
+    WebServicesTestUtils.checkStringMatch("hadoopBuildVersion",
         VersionInfo.getBuildVersion(), hadoopBuildVersion);
         VersionInfo.getBuildVersion(), hadoopBuildVersion);
     WebServicesTestUtils.checkStringMatch("hadoopVersion",
     WebServicesTestUtils.checkStringMatch("hadoopVersion",
         VersionInfo.getVersion(), hadoopVersion);
         VersionInfo.getVersion(), hadoopVersion);
 
 
     WebServicesTestUtils.checkStringMatch("resourceManagerVersionBuiltOn",
     WebServicesTestUtils.checkStringMatch("resourceManagerVersionBuiltOn",
         YarnVersionInfo.getDate(), resourceManagerVersionBuiltOn);
         YarnVersionInfo.getDate(), resourceManagerVersionBuiltOn);
-    WebServicesTestUtils.checkStringEqual("resourceManagerBuildVersion",
+    WebServicesTestUtils.checkStringMatch("resourceManagerBuildVersion",
         YarnVersionInfo.getBuildVersion(), resourceManagerBuildVersion);
         YarnVersionInfo.getBuildVersion(), resourceManagerBuildVersion);
     WebServicesTestUtils.checkStringMatch("resourceManagerVersion",
     WebServicesTestUtils.checkStringMatch("resourceManagerVersion",
         YarnVersionInfo.getVersion(), resourceManagerVersion);
         YarnVersionInfo.getVersion(), resourceManagerVersion);