Prechádzať zdrojové kódy

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 rokov pred
rodič
commit
43ac0739ce
57 zmenil súbory, kde vykonal 530 pridanie a 2502 odobranie
  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
     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
 
     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) {
         throw ioe;
       }
+
+      // otherwise back off and retry
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException ignored) {}
+      
       LOG.info("Retrying connect to server: " + server + ". Already tried "
           + curRetries + " time(s); maxRetries=" + maxRetries);
     }

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

@@ -4,10 +4,10 @@ Trunk (Unreleased)
 
   INCOMPATIBLE CHANGES
 
-    HDFS-4053. Increase the default block size. (eli)
-
     HDFS-3034. Remove the deprecated DFSOutputStream.sync() method.  (szetszwo)
 
+    HDFS-4434. Provide a mapping from INodeId to INode. (suresh)
+
   NEW FEATURES
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
@@ -15,9 +15,15 @@ Trunk (Unreleased)
     HDFS-3601. Add BlockPlacementPolicyWithNodeGroup to support block placement
     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.
     (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
     Azure environments. (See breakdown of tasks below for subtasks and
     contributors)
@@ -94,17 +100,38 @@ Trunk (Unreleased)
     HDFS-3476. Correct the default used in TestDFSClientRetries.busyTest()
     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-3498. Support replica removal in BlockPlacementPolicy and make
     BlockPlacementPolicyDefault extensible for reusing code in subclasses.
     (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-3768. Exception in TestJettyHelper is incorrect. 
     (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-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.
     (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.
     (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
     of persistent objects on heap. (harsh)
 
     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
     cache expires too quickly  (Chris Nauroth via Sanjay)
 
@@ -161,9 +214,14 @@ Trunk (Unreleased)
     HDFS-2776. Missing interface annotation on JournalSet. 
     (Brandon Li via jitendra)
 
+    HDFS-2908. Add apache license header for StorageReport.java. (Brandon Li
+    via jitendra)
+
     HDFS-3037. TestMulitipleNNDataBlockScanner#testBlockScannerAfterRestart is
     racy. (atm)
 
+    HDFS-2966. TestNameNodeMetrics tests can fail under load. (stevel)
+
     HDFS-3116. Typo in fetchdt error message. (AOE Takashi via atm)
 
     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.
     (Brandon Li via atm)
 
+    HDFS-3368. Missing blocks due to bad DataNodes coming up and down. (shv)
+
     HDFS-3462. TestDFSClientRetries.busyTest() should restore default
     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-3541. Deadlock between recovery, xceiver and packet responder (Vinay via umamahesh)
+
     HDFS-3482. hdfs balancer throws ArrayIndexOutOfBoundsException 
     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 
     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,
     the nodes in the same nodegroup should also be excluded.  (Junping Du
     via szetszwo)
@@ -232,6 +291,9 @@ Trunk (Unreleased)
     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)
 
+    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
     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-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
     certain order. (Andrew Wang via atm)
 
     HDFS-4586. TestDataDirs.testGetDataDirsFromURIs fails with all directories
     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
     failure to unpack old image tarball that contains hard links.
     (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
 
     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,
     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
 
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
@@ -365,26 +427,11 @@ Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
-    HDFS-4434. Provide a mapping from INodeId to INode. (suresh)
-
   NEW FEATURES
 
     HDFS-1804. Add a new block-volume device choosing policy that looks at
     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
 
     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 
     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
 
   BUG FIXES
@@ -474,9 +490,6 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4482. ReplicationMonitor thread can exit with NPE due to the race 
     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
     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. 
     (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.
     (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
     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
     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-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
 
   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
     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
   
     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
     via szetszwo)
 
+    HDFS-4053. Increase the default block size. (eli)
+
     HDFS-4088. Remove "throws QuotaExceededException" from an
     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 
     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)
 
@@ -768,7 +771,7 @@ Release 2.0.3-alpha - 2013-02-06
 
     HDFS-4031. Update findbugsExcludeFile.xml to include findbugs 2
     exclusions. (eli)
-
+    
     HDFS-4033. Miscellaneous findbugs 2 fixes. (eli)
 
     HDFS-4034. Remove redundant null checks. (eli)
@@ -845,6 +848,9 @@ Release 2.0.3-alpha - 2013-02-06
     HDFS-4006. TestCheckpoint#testSecondaryHasVeryOutOfDateImage
     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
     cluster cleanup. (eli)
 
@@ -892,6 +898,9 @@ Release 2.0.3-alpha - 2013-02-06
     HADOOP-8994. TestDFSShell creates file named "noFileHere", making further
     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
     (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
     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
     startup. (Junping Du via todd)
 
     HDFS-4138. BackupNode startup fails due to uninitialized edit log.
     (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 
     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.
     (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.
     (Eli Reisman via eli)
 
@@ -959,6 +965,10 @@ Release 2.0.3-alpha - 2013-02-06
 
     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
     storage edits. (todd)
 
@@ -1162,9 +1172,6 @@ Release 2.0.3-alpha - 2013-02-06
     HDFS-4351. In BlockPlacementPolicyDefault.chooseTarget(..), numOfReplicas
     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
     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.
     (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
     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
     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.
     (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.
     (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
     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
     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-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-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-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
     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 
     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
     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
     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
     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
     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-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
     missing range check. (eli)
 
-    HDFS-3541. Deadlock between recovery, xceiver and packet responder.
-    (Vinay via umamahesh)
-
     HDFS-3428. Move DelegationTokenRenewer to common (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.
     (Colin Patrick McCabe via eli)
 
-    HDFS-2966 TestNameNodeMetrics tests can fail under load. (stevel)
-
     HDFS-3605. Block mistakenly marked corrupt during edit log catchup
     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.
     (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
     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.
     (eli)
 
@@ -1734,16 +1690,9 @@ Release 2.0.2-alpha - 2012-09-07
     HDFS-3733. Audit logs should include WebHDFS access. (Andy Isaacson via 
     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.
     (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.
     (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-3833. TestDFSShell fails on windows due to concurrent file 
+    read/write. (Brandon Li via suresh)
+
     HDFS-3902. TestDatanodeBlockScanner#testBlockCorruptionPolicy is broken.
     (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 */
-  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) {
           // will not come here for root because root's nsQuota is always set
           return quotaNode.replaceSelf4INodeDirectory();
-          // update the inodeMap
-          inodeMap.put(newNode);
         }
       } else {
         // a non-quota directory; so replace it with a directory with quota
         return dirNode.replaceSelf4Quota(latest, nsQuota, dsQuota);
-        // update the inodeMap
-        inodeMap.put(newNode);
       }
       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();
       }
     });
-
-    // 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
     // keeping too many segments around.
     while (editLogs.size() > maxExtraEditsSegmentsToRetain) {
-      purgeLogsFrom = editLogs.get(0).getLastTxId() + 1;
+      purgeLogsFrom = editLogs.get(0).getFirstTxId();
       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();
       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
       // the xceiver to exit.
       assertXceiverCount(0);
@@ -198,7 +190,9 @@ public class TestDataTransferKeepalive {
   }
 
   private void assertXceiverCount(int expected) {
-    int count = getXceiverCountWithoutServer();
+    // Subtract 1, since the DataXceiverServer
+    // counts as one
+    int count = dn.getXceiverCount() - 1;
     if (count != expected) {
       ReflectionUtils.printThreadInfo(
           new PrintWriter(System.err),
@@ -207,14 +201,4 @@ public class TestDataTransferKeepalive {
           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;
 
 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.assertFalse;
 import static org.junit.Assert.fail;
 
 import java.io.FileNotFoundException;
@@ -863,39 +862,4 @@ public class TestINodeFile {
     resolvedPath = FSDirectory.resolvePath(testPath, components, fsd);
     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(201, 225), 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(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);
     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
     anyways after YARN-561. (Xuan Gong via vinodkv)
 
-    MAPREDUCE-5069. add concrete common implementations of
-    CombineFileInputFormat (Sangjin Lee via bobby)
-
   OPTIMIZATIONS
 
     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
     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
 
   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.impl.JobImpl;
 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.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.YarnException;
 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.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
@@ -276,7 +280,24 @@ public abstract class RMCommunicator extends AbstractService
       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>() {
       @Override
       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;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
@@ -90,6 +86,8 @@ public class LocalJobRunner implements ClientProtocol {
 
   private static final String jobDir =  "localRunner/";
 
+  private static final Counters EMPTY_COUNTERS = new Counters();
+
   public long getProtocolVersion(String protocol, long clientVersion) {
     return ClientProtocol.versionID;
   }
@@ -275,10 +273,10 @@ public class LocalJobRunner implements ClientProtocol {
       this.partialMapProgress = new float[numMaps];
       this.mapCounters = new Counters[numMaps];
       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,
         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());
       int taskIndex = mapIds.indexOf(taskId);
       if (taskIndex >= 0) {                       // mapping
@@ -536,10 +525,10 @@ public class LocalJobRunner implements ClientProtocol {
     public synchronized Counters getCurrentCounters() {
       if (null == mapCounters) {
         // Counters not yet initialized for job.
-        return new Counters();
+        return EMPTY_COUNTERS;
       }
 
-      Counters current = new Counters();
+      Counters current = EMPTY_COUNTERS;
       for (Counters c : mapCounters) {
         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);
       collector.flush();
-      
-      in.close();
-      in = null;
-      
-      collector.close();
-      collector = null;
     } 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(
               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>
@@ -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.Path;
 import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Writable;
@@ -429,15 +428,14 @@ public class ReduceTask extends Task {
     // make output collector
     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);
-    final RecordWriter<OUTKEY, OUTVALUE> finalOut = out;
-    
+
     OutputCollector<OUTKEY,OUTVALUE> collector = 
       new OutputCollector<OUTKEY,OUTVALUE>() {
         public void collect(OUTKEY key, OUTVALUE value)
           throws IOException {
-          finalOut.write(key, value);
+          out.write(key, value);
           // indicate that progress update needs to be sent
           reporter.progress();
         }
@@ -468,14 +466,20 @@ public class ReduceTask extends Task {
         values.informReduceProgress();
       }
 
+      //Clean up: repeated in catch block below
       reducer.close();
-      reducer = null;
-      
       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,
                                                reporter, comparator, keyClass,
                                                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 {
     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 {
     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
     return BuilderUtils.newApplicationReport(unknownAppId, unknownAttemptId,
         "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) {

+ 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",
         "appname", "host", 124, null, YarnApplicationState.FINISHED,
         "diagnostics", "url", 0, 0, FinalApplicationStatus.SUCCEEDED, null,
-        "N/A", 0.0f);
+        "N/A");
   }
 
   private ApplicationReport getRunningApplicationReport(String host, int port) {
@@ -423,7 +423,7 @@ public class TestClientServiceDelegate {
     return BuilderUtils.newApplicationReport(appId, attemptId, "user", "queue",
         "appname", host, port, null, YarnApplicationState.RUNNING,
         "diagnostics", "url", 0, 0, FinalApplicationStatus.UNDEFINED, null,
-        "N/A", 0.0f);
+        "N/A");
   }
 
   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
     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
 
     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
     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
 
   BUG FIXES
@@ -300,12 +290,6 @@ Release 2.0.5-beta - UNRELEASED
     YARN-549. YarnClient.submitApplication should wait for application to be
     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
 
   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 {
 
+  // 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.
   public static final String APPLICATION_CLIENT_SECRET_ENV_NAME =
     "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
   @Unstable
   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());
   }
 
-  @Override
-  public float getProgress() {
-    ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
-    return p.getProgress();
-  }
-
   @Override
   public void setApplicationId(ApplicationId applicationId) {
     maybeInitBuilder();
@@ -351,12 +345,6 @@ implements ApplicationReport {
     builder.setFinalApplicationStatus(convertToProtoFormat(finishState));
   }
 
-  @Override
-  public void setProgress(float progress) {
-    maybeInitBuilder();
-    builder.setProgress(progress);
-  }
-
   @Override
   public ApplicationReportProto getProto() {
     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 string originalTrackingUrl = 17;
   optional ApplicationAttemptIdProto currentApplicationAttemptId = 18;
-  optional float progress = 19;
 }
 
 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.conf.Configuration;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.YarnException;
 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.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
@@ -111,7 +115,24 @@ public class AMRMClientImpl extends AbstractService implements AMRMClient {
       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>() {
       @Override
       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.IOException;
 import java.io.PrintWriter;
-import java.text.DecimalFormat;
 import java.util.List;
 
 import org.apache.commons.cli.CommandLine;
@@ -35,7 +34,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 
 public class ApplicationCLI extends YarnCLI {
   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");
 
   public static void main(String[] args) throws Exception {
@@ -99,15 +98,12 @@ public class ApplicationCLI extends YarnCLI {
     writer.println("Total Applications:" + appsReport.size());
     writer.printf(APPLICATIONS_PATTERN, "Application-Id",
         "Application-Name", "User", "Queue", "State", "Final-State",
-        "Progress", "Tracking-URL");
+        "Tracking-URL");
     for (ApplicationReport appReport : appsReport) {
-      DecimalFormat formatter = new DecimalFormat("###.##%");
-      String progress = formatter.format(appReport.getProgress());
       writer.printf(APPLICATIONS_PATTERN, appReport.getApplicationId(),
           appReport.getName(), appReport.getUser(), appReport.getQueue(),
           appReport.getYarnApplicationState(), appReport
-              .getFinalApplicationStatus(),
-          progress, appReport.getOriginalTrackingUrl());
+              .getFinalApplicationStatus(), appReport.getOriginalTrackingUrl());
     }
     writer.flush();
   }
@@ -151,10 +147,6 @@ public class ApplicationCLI extends YarnCLI {
       appReportStr.println(appReport.getStartTime());
       appReportStr.print("\tFinish-Time : ");
       appReportStr.println(appReport.getFinishTime());
-      appReportStr.print("\tProgress : ");
-      DecimalFormat formatter = new DecimalFormat("###.##%");
-      String progress = formatter.format(appReport.getProgress());
-      appReportStr.println(progress);
       appReportStr.print("\tState : ");
       appReportStr.println(appReport.getYarnApplicationState());
       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),
         "user", "queue", "appname", "host", 124, null,
         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(
         newApplicationReport);
     int result = cli.run(new String[] { "-status", applicationId.toString() });
@@ -91,7 +91,6 @@ public class TestYarnCLI {
     pw.println("\tQueue : queue");
     pw.println("\tStart-Time : 0");
     pw.println("\tFinish-Time : 0");
-    pw.println("\tProgress : 53.79%");
     pw.println("\tState : FINISHED");
     pw.println("\tFinal-State : SUCCEEDED");
     pw.println("\tTracking-URL : N/A");
@@ -112,7 +111,7 @@ public class TestYarnCLI {
         applicationId, BuilderUtils.newApplicationAttemptId(applicationId, 1),
         "user", "queue", "appname", "host", 124, null,
         YarnApplicationState.FINISHED, "diagnostics", "url", 0, 0,
-        FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f);
+        FinalApplicationStatus.SUCCEEDED, null, "N/A");
     List<ApplicationReport> applicationReports = new ArrayList<ApplicationReport>();
     applicationReports.add(newApplicationReport);
     when(client.getApplicationList()).thenReturn(applicationReports);
@@ -125,12 +124,10 @@ public class TestYarnCLI {
     pw.println("Total Applications:1");
     pw.print("                Application-Id\t    Application-Name");
     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("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();
     String appsReportStr = baos.toString("UTF-8");
     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,
       String url, long startTime, long finishTime,
       FinalApplicationStatus finalStatus,
-      ApplicationResourceUsageReport appResources, String origTrackingUrl,
-      float progress) {
+      ApplicationResourceUsageReport appResources, String origTrackingUrl) {
     ApplicationReport report = recordFactory
         .newRecordInstance(ApplicationReport.class);
     report.setApplicationId(applicationId);
@@ -353,7 +352,6 @@ public class BuilderUtils {
     report.setFinalApplicationStatus(finalStatus);
     report.setApplicationResourceUsageReport(appResources);
     report.setOriginalTrackingUrl(origTrackingUrl);
-    report.setProgress(progress);
     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));
   }
 
-  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",
         VersionInfo.getDate(), hadoopVersionBuiltOn);
-    WebServicesTestUtils.checkStringEqual("hadoopBuildVersion",
+    WebServicesTestUtils.checkStringMatch("hadoopBuildVersion",
         VersionInfo.getBuildVersion(), hadoopBuildVersion);
     WebServicesTestUtils.checkStringMatch("hadoopVersion",
         VersionInfo.getVersion(), hadoopVersion);
 
     WebServicesTestUtils.checkStringMatch("resourceManagerVersionBuiltOn",
         YarnVersionInfo.getDate(), resourceManagerVersionBuiltOn);
-    WebServicesTestUtils.checkStringEqual("resourceManagerBuildVersion",
+    WebServicesTestUtils.checkStringMatch("resourceManagerBuildVersion",
         YarnVersionInfo.getBuildVersion(), resourceManagerBuildVersion);
     WebServicesTestUtils.checkStringMatch("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(
           application.getAppAttemptId());
-      Token<ApplicationTokenIdentifier> appMasterToken =
+      Token<ApplicationTokenIdentifier> token =
           new Token<ApplicationTokenIdentifier>(id,
               this.rmContext.getApplicationTokenSecretManager());
       InetSocketAddress serviceAddr = conf.getSocketAddr(
@@ -212,11 +212,16 @@ public class AMLauncher implements Runnable {
           YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
           YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
       // 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();
       credentials.writeTokenStorageToStream(dob);
       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) {
     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;
       FinalApplicationStatus finishState = getFinalApplicationStatus();
       String diags = UNAVAILABLE;
-      float progress = 0.0f;
       if (allowAccess) {
         if (this.currentAttempt != null) {
           currentApplicationAttemptId = this.currentAttempt.getAppAttemptId();
@@ -447,8 +446,8 @@ public class RMAppImpl implements RMApp, Recoverable {
           host = this.currentAttempt.getHost();
           rpcPort = this.currentAttempt.getRpcPort();
           appUsageReport = currentAttempt.getApplicationResourceUsageReport();
-          progress = currentAttempt.getProgress();
         }
+
         diags = this.diagnostics.toString();
       }
 
@@ -463,7 +462,7 @@ public class RMAppImpl implements RMApp, Recoverable {
           this.name, host, rpcPort, clientToken,
           createApplicationState(this.stateMachine.getCurrentState()), diags,
           trackingUrl, this.startTime, this.finishTime, finishState,
-          appUsageReport, origTrackingUrl, progress);
+          appUsageReport, origTrackingUrl);
     } finally {
       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.factories.RecordFactory;
 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.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
@@ -46,9 +45,6 @@ import org.apache.hadoop.yarn.util.BuilderUtils;
 @Private
 @Unstable
 public class AppSchedulable extends Schedulable {
-  private static final DefaultResourceCalculator RESOURCE_CALCULATOR
-    = new DefaultResourceCalculator();
-  
   private FairScheduler scheduler;
   private FSSchedulerApp app;
   private Resource demand = Resources.createResource(0);
@@ -184,15 +180,15 @@ public class AppSchedulable extends Schedulable {
    * update relevant bookeeping. This dispatches ro relevant handlers
    * 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() +
                                  " app_id=" + app.getApplicationId());
     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);
-      node.reserveResource(app, priority, rmContainer);
+      node.reserveResource(application, priority, rmContainer);
       getMetrics().reserveResource(app.getUser(),
           container.getResource());
       scheduler.getRootQueueMetrics().reserveResource(app.getUser(),
@@ -201,24 +197,25 @@ public class AppSchedulable extends Schedulable {
 
     else {
       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
    * handlers for an unreservation.
    */
-  public void unreserve(Priority priority, FSSchedulerNode node) {
+  private void unreserve(FSSchedulerApp application, Priority priority,
+      FSSchedulerNode node) {
     RMContainer rmContainer = node.getReservedContainer();
-    app.unreserve(node, priority);
-    node.unreserveResource(app);
+    application.unreserve(node, priority);
+    node.unreserveResource(application);
     getMetrics().unreserveResource(
-        app.getUser(), rmContainer.getContainer().getResource());
+        application.getUser(), rmContainer.getContainer().getResource());
     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.
    */
   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?
     Resource capability = request.getCapability();
@@ -240,7 +237,7 @@ public class AppSchedulable extends Schedulable {
     if (reserved) {
       container = node.getReservedContainer().getContainer();
     } else {
-      container = createContainer(app, node, capability, priority);
+      container = createContainer(application, node, capability, priority);
     }
 
     // Can we allocate a container on this node?
@@ -250,12 +247,9 @@ public class AppSchedulable extends Schedulable {
     if (availableContainers > 0) {
       // Inform the application of the new container for this request
       RMContainer allocatedContainer =
-          app.allocate(type, node, priority, request, container);
+          application.allocate(type, node, priority, request, container);
       if (allocatedContainer == null) {
         // Did the application need this resource?
-        if (reserved) {
-          unreserve(priority, node);
-        }
         return Resources.none();
       }
       else {
@@ -268,17 +262,17 @@ public class AppSchedulable extends Schedulable {
 
       // If we had previously made a reservation, delete it
       if (reserved) {
-        unreserve(priority, node);
+        unreserve(application, priority, node);
       }
 
       // Inform the node
-      node.allocateContainer(app.getApplicationId(),
+      node.allocateContainer(application.getApplicationId(),
           allocatedContainer);
 
       return container.getResource();
     } else {
       // The desired container won't fit here, so reserve
-      reserve(priority, node, container, reserved);
+      reserve(application, priority, node, 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
       if (app.getTotalRequiredResources(priority) == 0) {
-        unreserve(priority, node);
+        unreserve(app, priority, node);
         return Resources.none();
       }
     } else {
@@ -310,8 +304,7 @@ public class AppSchedulable extends Schedulable {
     // (not scheduled) in order to promote better locality.
     synchronized (app) {
       for (Priority priority : prioritiesToTry) {
-        if (app.getTotalRequiredResources(priority) <= 0 ||
-            !hasContainerForNode(priority, node)) {
+        if (app.getTotalRequiredResources(priority) <= 0) {
           continue;
         }
         
@@ -328,14 +321,14 @@ public class AppSchedulable extends Schedulable {
         
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
             && localRequest != null && localRequest.getNumContainers() != 0) {
-          return assignContainer(node, priority,
+          return assignContainer(node, app, priority,
               localRequest, NodeType.NODE_LOCAL, reserved);
         }
 
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
             && (allowedLocality.equals(NodeType.RACK_LOCAL) ||
                 allowedLocality.equals(NodeType.OFF_SWITCH))) {
-          return assignContainer(node, priority, rackLocalRequest,
+          return assignContainer(node, app, priority, rackLocalRequest,
               NodeType.RACK_LOCAL, reserved);
         }
 
@@ -343,7 +336,7 @@ public class AppSchedulable extends Schedulable {
             ResourceRequest.ANY);
         if (offSwitchRequest != null && offSwitchRequest.getNumContainers() != 0
             && allowedLocality.equals(NodeType.OFF_SWITCH)) {
-          return assignContainer(node, priority, offSwitchRequest,
+          return assignContainer(node, app, priority, offSwitchRequest,
               NodeType.OFF_SWITCH, reserved);
         }
       }
@@ -359,16 +352,4 @@ public class AppSchedulable extends Schedulable {
   public Resource assignContainer(FSSchedulerNode node) {
     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.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 public class FSLeafQueue extends FSQueue {
   private static final Log LOG = LogFactory.getLog(
@@ -127,8 +126,8 @@ public class FSLeafQueue extends FSQueue {
             + demand);
       }
       demand = Resources.add(demand, toAdd);
-      demand = Resources.componentwiseMin(demand, maxRes);
-      if (Resources.equals(demand, maxRes)) {
+      if (Resources.greaterThanOrEqual(demand, maxRes)) {
+        demand = maxRes;
         break;
       }
     }
@@ -154,7 +153,7 @@ public class FSLeafQueue extends FSQueue {
     for (AppSchedulable sched : appScheds) {
       if (sched.getRunnable()) {
         assigned = sched.assignContainer(node);
-        if (!assigned.equals(Resources.none())) {
+        if (Resources.greaterThan(assigned, Resources.none())) {
           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.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 
 public class FSParentQueue extends FSQueue {
   private static final Log LOG = LogFactory.getLog(
@@ -88,8 +87,8 @@ public class FSParentQueue extends FSQueue {
             " now " + demand);
       }
       demand = Resources.add(demand, toAdd);
-      demand = Resources.componentwiseMin(demand, maxRes);
-      if (Resources.equals(demand, maxRes)) {
+      if (Resources.greaterThanOrEqual(demand, maxRes)) {
+        demand = maxRes;
         break;
       }
     }
@@ -136,14 +135,16 @@ public class FSParentQueue extends FSQueue {
     Resource assigned = Resources.none();
 
     // If this queue is over its limit, reject
-    if (!assignContainerPreCheck(node)) {
+    if (Resources.greaterThan(getResourceUsage(),
+        queueMgr.getMaxResources(getName()))) {
       return assigned;
     }
 
     Collections.sort(childQueues, policy.getComparator());
     for (FSQueue child : childQueues) {
       assigned = child.assignContainer(node);
-      if (!Resources.equals(assigned, Resources.none())) {
+      if (node.getReservedContainer() != null
+          || Resources.greaterThan(assigned, Resources.none())) {
         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.factories.RecordFactory;
 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.QueueMetrics;
 
@@ -162,7 +161,7 @@ public abstract class FSQueue extends Schedulable implements Queue {
    * @return true if check passes (can assign) or false otherwise
    */
   protected boolean assignContainerPreCheck(FSSchedulerNode node) {
-    if (!Resources.fitsIn(getResourceUsage(),
+    if (Resources.greaterThan(getResourceUsage(),
         queueMgr.getMaxResources(getName()))
         || node.getReservedContainer() != null) {
       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
       .getRecordFactory(null);
 
-  private Resource availableResource;
+  private Resource availableResource = recordFactory.newRecordInstance(Resource.class);
   private Resource usedResource = recordFactory.newRecordInstance(Resource.class);
 
   private volatile int numContainers;
@@ -62,7 +62,7 @@ public class FSSchedulerNode extends SchedulerNode {
 
   public FSSchedulerNode(RMNode node) {
     this.rmNode = node;
-    this.availableResource = Resources.clone(node.getTotalCapability());
+    this.availableResource.setMemory(node.getTotalCapability().getMemory());
   }
 
   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.ContainerStatus;
 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.QueueInfo;
 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.RMContext;
 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.RMAppAttemptEventType;
 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 ResourceCalculator RESOURCE_CALCULATOR =
-      new DefaultResourceCalculator();
-  
   // Value that container assignment methods return when a container is
   // reserved
   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?
    */
   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.
    */
   boolean isStarvedForFairShare(FSLeafQueue sched) {
-    Resource desiredFairShare = Resources.max(RESOURCE_CALCULATOR, clusterCapacity,
+    Resource desiredFairShare = Resources.max(
         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()) {
       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);
     }
   }
@@ -320,8 +309,7 @@ public class FairScheduler implements ResourceScheduler {
     // Collect running containers from over-scheduled queues
     List<RMContainer> runningContainers = new ArrayList<RMContainer>();
     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 (RMContainer c : as.getApp().getLiveContainers()) {
             runningContainers.add(c);
@@ -344,8 +332,7 @@ public class FairScheduler implements ResourceScheduler {
     // tasks, making sure we don't kill too many from any queue
     for (RMContainer container : runningContainers) {
       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() +
             "res=" + container.getContainer().getResource() +
             ") from queue " + sched.getName());
@@ -358,8 +345,7 @@ public class FairScheduler implements ResourceScheduler {
 
         toPreempt = Resources.subtract(toPreempt,
             container.getContainer().getResource());
-        if (Resources.lessThanOrEqual(RESOURCE_CALCULATOR, clusterCapacity,
-            toPreempt, Resources.none())) {
+        if (Resources.equals(toPreempt, Resources.none())) {
           break;
         }
       }
@@ -383,21 +369,17 @@ public class FairScheduler implements ResourceScheduler {
     Resource resDueToMinShare = Resources.none();
     Resource resDueToFairShare = Resources.none();
     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) {
-      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 "
           + sched.getName() + ": resDueToMinShare = " + resDueToMinShare
           + ", resDueToFairShare = " + resDueToFairShare;
@@ -806,32 +788,21 @@ public class FairScheduler implements ResourceScheduler {
 
     AppSchedulable reservedAppSchedulable = node.getReservedAppSchedulable();
     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
       int assignedContainers = 0;
       while (node.getReservedContainer() == null) {
         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;
         }
         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.Resource;
 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.Element;
 import org.w3c.dom.Node;
@@ -475,8 +474,8 @@ public class QueueManager {
     }
     queueAcls.put(queueName, acls);
     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",
           queueName, maxQueueResources.get(queueName), minQueueResources.get(queueName)));
     }
@@ -505,7 +504,7 @@ public class QueueManager {
     if (maxQueueResource != null) {
       return maxQueueResource;
     } 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 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.SchedulingPolicy;
 
@@ -32,8 +31,6 @@ import com.google.common.annotations.VisibleForTesting;
 public class FairSharePolicy extends SchedulingPolicy {
   @VisibleForTesting
   public static final String NAME = "Fairshare";
-  private static final DefaultResourceCalculator RESOURCE_CALCULATOR =
-      new DefaultResourceCalculator();
   private FairShareComparator comparator = new FairShareComparator();
 
   @Override
@@ -62,19 +59,15 @@ public class FairSharePolicy extends SchedulingPolicy {
     public int compare(Schedulable s1, Schedulable s2) {
       double minShareRatio1, minShareRatio2;
       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);
       minShareRatio1 = (double) s1.getResourceUsage().getMemory()
-          / Resources.max(RESOURCE_CALCULATOR, null, minShare1, one).getMemory();
+          / Resources.max(minShare1, one).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();
       useToWeightRatio2 = s2.getResourceUsage().getMemory() / s2.getWeight();
       int res = 0;
@@ -168,11 +161,9 @@ public class FairSharePolicy extends SchedulingPolicy {
     for (Schedulable sched : schedulables) {
       Resources.addTo(totalDemand, sched.getDemand());
     }
-    Resource cap = Resources.min(RESOURCE_CALCULATOR, null, totalDemand,
-        totalResources);
+    Resource cap = Resources.min(totalDemand, totalResources);
     double rMax = 1.0;
-    while (Resources.lessThan(RESOURCE_CALCULATOR, null,
-        resUsedWithWeightToResRatio(rMax, schedulables),
+    while (Resources.lessThan(resUsedWithWeightToResRatio(rMax, schedulables),
         cap)) {
       rMax *= 2.0;
     }
@@ -181,8 +172,7 @@ public class FairSharePolicy extends SchedulingPolicy {
     double right = rMax;
     for (int i = 0; i < COMPUTE_FAIR_SHARES_ITERATIONS; i++) {
       double mid = (left + right) / 2.0;
-      if (Resources.lessThan(RESOURCE_CALCULATOR, null,
-          resUsedWithWeightToResRatio(mid, schedulables),
+      if (Resources.lessThan(resUsedWithWeightToResRatio(mid, schedulables),
           cap)) {
         left = mid;
       } 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 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.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
 
 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.net.InetSocketAddress;
-import java.nio.ByteBuffer;
 import java.security.PrivilegedAction;
 import java.util.HashMap;
 import java.util.Map;
@@ -29,10 +28,11 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 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.protocolrecords.GetContainerStatusRequest;
 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 Configuration confWithSecurityEnabled =
-      new Configuration();
-  static {
-    confWithSecurityEnabled.set(
-      CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(confWithSecurityEnabled);
-  }
-
   public static final class MyContainerManager implements ContainerManager {
 
-    public ByteBuffer amTokens;
+    public Map<String, String> amContainerEnv;
 
     public MyContainerManager() {
     }
@@ -78,7 +70,7 @@ public class TestAMAuthorization {
     public StartContainerResponse
         startContainer(StartContainerRequest request)
             throws YarnRemoteException {
-      amTokens = request.getContainerLaunchContext().getContainerTokens();
+      amContainerEnv = request.getContainerLaunchContext().getEnvironment();
       return null;
     }
 
@@ -101,6 +93,9 @@ public class TestAMAuthorization {
 
     public MockRMWithAMS(Configuration conf, ContainerManager containerManager) {
       super(conf, containerManager);
+      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+          "kerberos");
+      UserGroupInformation.setConfiguration(conf);
     }
 
     @Override
@@ -110,6 +105,7 @@ public class TestAMAuthorization {
 
     @Override
     protected ApplicationMasterService createApplicationMasterService() {
+
       return new ApplicationMasterService(getRMContext(), this.scheduler);
     }
   }
@@ -117,8 +113,7 @@ public class TestAMAuthorization {
   @Test
   public void testAuthorizedAccess() throws Exception {
     MyContainerManager containerManager = new MyContainerManager();
-    final MockRM rm =
-        new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+    final MockRM rm = new MockRMWithAMS(new Configuration(), containerManager);
     rm.start();
 
     MockNM nm1 = rm.registerNode("localhost:1234", 5120);
@@ -131,11 +126,11 @@ public class TestAMAuthorization {
     nm1.nodeHeartbeat(true);
 
     int waitCount = 0;
-    while (containerManager.amTokens == null && waitCount++ < 20) {
+    while (containerManager.amContainerEnv == null && waitCount++ < 20) {
       LOG.info("Waiting for AM Launch to happen..");
       Thread.sleep(1000);
     }
-    Assert.assertNotNull(containerManager.amTokens);
+    Assert.assertNotNull(containerManager.amContainerEnv);
 
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -147,12 +142,12 @@ public class TestAMAuthorization {
 
     UserGroupInformation currentUser = UserGroupInformation
         .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
         .doAs(new PrivilegedAction<AMRMProtocol>() {
@@ -177,7 +172,7 @@ public class TestAMAuthorization {
   @Test
   public void testUnauthorizedAccess() throws Exception {
     MyContainerManager containerManager = new MyContainerManager();
-    MockRM rm = new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+    MockRM rm = new MockRMWithAMS(new Configuration(), containerManager);
     rm.start();
 
     MockNM nm1 = rm.registerNode("localhost:1234", 5120);
@@ -187,16 +182,17 @@ public class TestAMAuthorization {
     nm1.nodeHeartbeat(true);
 
     int waitCount = 0;
-    while (containerManager.amTokens == null && waitCount++ < 20) {
+    while (containerManager.amContainerEnv == null && waitCount++ < 20) {
       LOG.info("Waiting for AM Launch to happen..");
       Thread.sleep(1000);
     }
-    Assert.assertNotNull(containerManager.amTokens);
+    Assert.assertNotNull(containerManager.amContainerEnv);
 
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
     waitForLaunchedState(attempt);
 
+    // Create a client to the RM.
     final Configuration conf = rm.getConfig();
     final YarnRPC rpc = YarnRPC.create(conf);
     final InetSocketAddress serviceAddr = conf.getSocketAddr(
@@ -206,8 +202,13 @@ public class TestAMAuthorization {
 
     UserGroupInformation currentUser = UserGroupInformation
         .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
         .doAs(new PrivilegedAction<AMRMProtocol>() {
           @Override
@@ -216,39 +217,9 @@ public class TestAMAuthorization {
                 serviceAddr, conf);
           }
         });
+
     RegisterApplicationMasterRequest request = Records
         .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
         .newApplicationAttemptId(applicationAttemptId.getApplicationId(), 42);
     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());
   }
-  
-  @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.LogFactory;
 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.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 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.FinishApplicationMasterRequest;
 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 Configuration confWithSecurityEnabled =
-      new Configuration();
-  static {
-    confWithSecurityEnabled.set(
-      CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(confWithSecurityEnabled);
-  }
-
   /**
    * Validate that application tokens are unusable after the
    * application-finishes.
@@ -69,8 +61,7 @@ public class TestApplicationTokens {
   public void testTokenExpiry() throws Exception {
 
     MyContainerManager containerManager = new MyContainerManager();
-    final MockRM rm =
-        new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+    final MockRM rm = new MockRMWithAMS(new Configuration(), containerManager);
     rm.start();
 
     final Configuration conf = rm.getConfig();
@@ -85,11 +76,11 @@ public class TestApplicationTokens {
       nm1.nodeHeartbeat(true);
 
       int waitCount = 0;
-      while (containerManager.amTokens == null && waitCount++ < 20) {
+      while (containerManager.amContainerEnv == null && waitCount++ < 20) {
         LOG.info("Waiting for AM Launch to happen..");
         Thread.sleep(1000);
       }
-      Assert.assertNotNull(containerManager.amTokens);
+      Assert.assertNotNull(containerManager.amContainerEnv);
 
       RMAppAttempt attempt = app.getCurrentAppAttempt();
       ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -98,12 +89,13 @@ public class TestApplicationTokens {
       UserGroupInformation currentUser =
           UserGroupInformation
             .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);
 
@@ -160,9 +152,9 @@ public class TestApplicationTokens {
   @Test
   public void testMasterKeyRollOver() throws Exception {
 
+    Configuration config = new Configuration();
     MyContainerManager containerManager = new MyContainerManager();
-    final MockRM rm =
-        new MockRMWithAMS(confWithSecurityEnabled, containerManager);
+    final MockRM rm = new MockRMWithAMS(config, containerManager);
     rm.start();
 
     final Configuration conf = rm.getConfig();
@@ -177,11 +169,11 @@ public class TestApplicationTokens {
       nm1.nodeHeartbeat(true);
 
       int waitCount = 0;
-      while (containerManager.amTokens == null && waitCount++ < 20) {
+      while (containerManager.amContainerEnv == null && waitCount++ < 20) {
         LOG.info("Waiting for AM Launch to happen..");
         Thread.sleep(1000);
       }
-      Assert.assertNotNull(containerManager.amTokens);
+      Assert.assertNotNull(containerManager.amContainerEnv);
 
       RMAppAttempt attempt = app.getCurrentAppAttempt();
       ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId();
@@ -190,12 +182,13 @@ public class TestApplicationTokens {
       UserGroupInformation currentUser =
           UserGroupInformation
             .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);
 

+ 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",
         VersionInfo.getDate(), hadoopVersionBuiltOn);
-    WebServicesTestUtils.checkStringEqual("hadoopBuildVersion",
+    WebServicesTestUtils.checkStringMatch("hadoopBuildVersion",
         VersionInfo.getBuildVersion(), hadoopBuildVersion);
     WebServicesTestUtils.checkStringMatch("hadoopVersion",
         VersionInfo.getVersion(), hadoopVersion);
 
     WebServicesTestUtils.checkStringMatch("resourceManagerVersionBuiltOn",
         YarnVersionInfo.getDate(), resourceManagerVersionBuiltOn);
-    WebServicesTestUtils.checkStringEqual("resourceManagerBuildVersion",
+    WebServicesTestUtils.checkStringMatch("resourceManagerBuildVersion",
         YarnVersionInfo.getBuildVersion(), resourceManagerBuildVersion);
     WebServicesTestUtils.checkStringMatch("resourceManagerVersion",
         YarnVersionInfo.getVersion(), resourceManagerVersion);