From fa500825fab5fcdba20cb2b940c9ee94ccdfa1dd Mon Sep 17 00:00:00 2001 From: Tsz-wo Sze Date: Fri, 26 Apr 2013 01:19:00 +0000 Subject: [PATCH] Merge r1471229 through r1476009 from trunk. git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1476011 13f79535-47bb-0310-9956-ffa450edef68 --- .../hadoop-common/CHANGES.txt | 3 + .../java/org/apache/hadoop/ipc/Client.java | 6 - hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 270 ++++++++------ .../hdfs/server/namenode/FSDirectory.java | 10 +- .../namenode/NNStorageRetentionManager.java | 8 +- .../hdfs/TestDataTransferKeepalive.java | 24 +- .../hdfs/server/namenode/TestINodeFile.java | 38 +- .../TestNNStorageRetentionManager.java | 7 +- hadoop-mapreduce-project/CHANGES.txt | 20 ++ .../mapreduce/v2/app/rm/RMCommunicator.java | 23 +- .../apache/hadoop/mapred/LocalJobRunner.java | 23 +- .../org/apache/hadoop/mapred/MapTask.java | 83 ++++- .../org/apache/hadoop/mapred/ReduceTask.java | 44 ++- .../lib/CombineFileRecordReaderWrapper.java | 86 +++++ .../lib/CombineSequenceFileInputFormat.java | 66 ++++ .../mapred/lib/CombineTextInputFormat.java | 68 ++++ .../org/apache/hadoop/mapreduce/Mapper.java | 9 +- .../org/apache/hadoop/mapreduce/Reducer.java | 17 +- .../input/CombineFileRecordReaderWrapper.java | 108 ++++++ .../input/CombineSequenceFileInputFormat.java | 64 ++++ .../lib/input/CombineTextInputFormat.java | 65 ++++ .../apache/hadoop/mapred/NotRunningJob.java | 2 +- .../mapred/TestClientServiceDelegate.java | 4 +- .../TestCombineSequenceFileInputFormat.java | 170 +++++++++ .../mapred/TestCombineTextInputFormat.java | 250 +++++++++++++ .../mapreduce/TestMapperReducerCleanup.java | 334 ++++++++++++++++++ .../TestCombineSequenceFileInputFormat.java | 186 ++++++++++ .../lib/input/TestCombineTextInputFormat.java | 267 ++++++++++++++ hadoop-yarn-project/CHANGES.txt | 16 + .../hadoop/yarn/api/ApplicationConstants.java | 4 - .../yarn/api/records/ApplicationReport.java | 12 + .../impl/pb/ApplicationReportPBImpl.java | 12 + .../src/main/proto/yarn_protos.proto | 1 + .../hadoop/yarn/client/AMRMClientImpl.java | 23 +- .../yarn/client/cli/ApplicationCLI.java | 14 +- .../hadoop/yarn/client/cli/TestYarnCLI.java | 11 +- .../apache/hadoop/yarn/util/BuilderUtils.java | 4 +- .../yarn/webapp/WebServicesTestUtils.java | 5 + .../nodemanager/webapp/TestNMWebServices.java | 4 +- .../amlauncher/AMLauncher.java | 15 +- .../resourcemanager/resource/Resources.java | 10 + .../resourcemanager/rmapp/RMAppImpl.java | 5 +- .../scheduler/fair/AppSchedulable.java | 71 ++-- .../scheduler/fair/FSLeafQueue.java | 7 +- .../scheduler/fair/FSParentQueue.java | 11 +- .../scheduler/fair/FSQueue.java | 3 +- .../scheduler/fair/FSSchedulerNode.java | 4 +- .../scheduler/fair/FairScheduler.java | 79 +++-- .../scheduler/fair/QueueManager.java | 7 +- .../scheduler/fair/Resources.java | 150 -------- .../fair/policies/FairSharePolicy.java | 30 +- .../scheduler/fair/policies/FifoPolicy.java | 2 +- .../resourcemanager/TestAMAuthorization.java | 87 +++-- .../resource/TestResources.java | 43 +++ .../scheduler/fair/TestFairScheduler.java | 24 ++ .../security/TestApplicationTokens.java | 55 +-- .../webapp/TestRMWebServices.java | 4 +- 57 files changed, 2442 insertions(+), 526 deletions(-) create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReaderWrapper.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineSequenceFileInputFormat.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineTextInputFormat.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileRecordReaderWrapper.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineSequenceFileInputFormat.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineTextInputFormat.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapperReducerCleanup.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java delete mode 100644 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 create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResources.java diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index a0bcf71e38b..47968d336ce 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -559,6 +559,9 @@ 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 diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java index 986b1a2c154..b6ffda82213 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -706,12 +706,6 @@ 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); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index b46d20f4e26..4d77689d9e9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -4,9 +4,9 @@ Trunk (Unreleased) INCOMPATIBLE CHANGES - HDFS-3034. Remove the deprecated DFSOutputStream.sync() method. (szetszwo) + HDFS-4053. Increase the default block size. (eli) - HDFS-4434. Provide a mapping from INodeId to INode. (suresh) + HDFS-3034. Remove the deprecated DFSOutputStream.sync() method. (szetszwo) NEW FEATURES @@ -15,15 +15,9 @@ 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) @@ -100,38 +94,17 @@ 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) @@ -152,40 +125,14 @@ 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) @@ -214,14 +161,9 @@ 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 @@ -241,8 +183,6 @@ 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) @@ -250,8 +190,6 @@ 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) @@ -284,6 +222,9 @@ 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) @@ -291,9 +232,6 @@ 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) @@ -311,25 +249,19 @@ 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, @@ -367,6 +299,12 @@ 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. @@ -427,11 +365,26 @@ 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 @@ -473,6 +426,37 @@ 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 @@ -490,6 +474,9 @@ 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) @@ -546,12 +533,6 @@ 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) @@ -563,6 +544,9 @@ 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) @@ -582,6 +566,12 @@ 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 @@ -634,6 +624,10 @@ 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 @@ -691,8 +685,6 @@ 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) @@ -708,12 +700,17 @@ 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) + HDFS-3573. Supply NamespaceInfo when instantiating JournalManagers. + (todd and ivank via umamahesh) - HDFS-3695. Genericize format() to non-file JournalManagers. (todd) + HDFS-3695. Genericize format() to non-file JournalManagers. + (todd via umamahesh) - HDFS-3789. JournalManager#format() should be able to throw IOException - (Ivan Kelly via 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-3916. libwebhdfs testing code cleanup. (Jing Zhao via suresh) @@ -771,7 +768,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) @@ -848,9 +845,6 @@ 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) @@ -898,9 +892,6 @@ 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) @@ -912,14 +903,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) @@ -946,6 +937,9 @@ 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) @@ -965,10 +959,6 @@ 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) @@ -1172,6 +1162,9 @@ 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) @@ -1209,9 +1202,11 @@ 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) + HDFS-3390. DFSAdmin should print full stack traces of errors when DEBUG logging is enabled. (atm) @@ -1385,6 +1380,12 @@ 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) @@ -1399,13 +1400,20 @@ 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-2686. Remove DistributedUpgrade related code. (suresh) - HDFS-3832. Remove protocol methods related to DistributedUpgrade. (suresh) + HDFS-3819. Should check whether invalidate work percentage default value is + not greater than 1.0f. (Jing Zhao via jitendra) + 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) @@ -1416,6 +1424,9 @@ 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 @@ -1475,6 +1486,9 @@ 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) @@ -1485,6 +1499,28 @@ 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) @@ -1497,6 +1533,9 @@ 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) @@ -1529,7 +1568,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) @@ -1562,6 +1601,9 @@ 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) @@ -1589,6 +1631,8 @@ 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) @@ -1667,12 +1711,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) @@ -1690,9 +1734,16 @@ 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) @@ -1736,9 +1787,6 @@ 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) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java index 8e7043ce8cc..2a62994e306 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java @@ -2349,11 +2349,17 @@ public class FSDirectory implements Closeable { counts.get(Quota.DISKSPACE)); } else if (!quotaNode.isQuotaSet() && latest == null) { // will not come here for root because root's nsQuota is always set - return quotaNode.replaceSelf4INodeDirectory(); + INodeDirectory newNode = quotaNode.replaceSelf4INodeDirectory(); + // update the inodeMap + inodeMap.put(newNode); + return newNode; } } else { // a non-quota directory; so replace it with a directory with quota - return dirNode.replaceSelf4Quota(latest, nsQuota, dsQuota); + INodeDirectory newNode = dirNode.replaceSelf4Quota(latest, nsQuota, dsQuota); + // update the inodeMap + inodeMap.put(newNode); + return newNode; } return (oldNsQuota != nsQuota || oldDsQuota != dsQuota) ? dirNode : null; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java index 8c51a99a36a..75d1bd82fec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java @@ -118,11 +118,17 @@ 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).getFirstTxId(); + purgeLogsFrom = editLogs.get(0).getLastTxId() + 1; editLogs.remove(0); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java index 3c9ee25b111..bf4e13bd027 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java @@ -146,7 +146,15 @@ public class TestDataTransferKeepalive { stm.read(); assertXceiverCount(1); - Thread.sleep(WRITE_TIMEOUT + 1000); + // 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; + } + // DN should time out in sendChunks, and this should force // the xceiver to exit. assertXceiverCount(0); @@ -190,9 +198,7 @@ public class TestDataTransferKeepalive { } private void assertXceiverCount(int expected) { - // Subtract 1, since the DataXceiverServer - // counts as one - int count = dn.getXceiverCount() - 1; + int count = getXceiverCountWithoutServer(); if (count != expected) { ReflectionUtils.printThreadInfo( new PrintWriter(System.err), @@ -201,4 +207,14 @@ 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; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java index 2701d772e4c..773f3ade3d8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java @@ -19,8 +19,9 @@ package org.apache.hadoop.hdfs.server.namenode; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.FileNotFoundException; @@ -862,4 +863,39 @@ 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); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java index 4c7662efdba..161a9f1fe84 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java @@ -217,11 +217,12 @@ 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-400 segment is considered required, - // not extra. + // Only retain 2 extra segments. The 301-350 and 351-400 segments are + // 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, 400), false); + tc.addLog("/foo2/current/" + getFinalizedEditsFileName(301, 350), false); + tc.addLog("/foo2/current/" + getFinalizedEditsFileName(351, 400), false); tc.addLog("/foo2/current/" + getInProgressEditsFileName(401), false); runTest(tc); } diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index 51c11136754..e18cd9fb5f9 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -209,6 +209,9 @@ 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 @@ -336,6 +339,23 @@ 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 diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java index 1abbc7159ff..0e4cfe05857 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java @@ -38,13 +38,9 @@ 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; @@ -280,24 +276,7 @@ public abstract class RMCommunicator extends AbstractService throw new YarnException(e); } - if (UserGroupInformation.isSecurityEnabled()) { - String tokenURLEncodedStr = System.getenv().get( - ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME); - Token token = new Token(); - - 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); - } - + // CurrentUser should already have AMToken loaded. return currentUser.doAs(new PrivilegedAction() { @Override public AMRMProtocol run() { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java index 85f8c183fa3..d71aff518d0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java @@ -18,6 +18,10 @@ 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; @@ -86,8 +90,6 @@ 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; } @@ -273,10 +275,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] = EMPTY_COUNTERS; + this.mapCounters[i] = new Counters(); } - this.reduceCounters = EMPTY_COUNTERS; + this.reduceCounters = new Counters(); } /** @@ -497,6 +499,15 @@ 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 @@ -525,10 +536,10 @@ public class LocalJobRunner implements ClientProtocol { public synchronized Counters getCurrentCounters() { if (null == mapCounters) { // Counters not yet initialized for job. - return EMPTY_COUNTERS; + return new Counters(); } - Counters current = EMPTY_COUNTERS; + Counters current = new Counters(); for (Counters c : mapCounters) { current = Counters.sum(current, c); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java index cd4ba51c31c..84404d1ef13 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java @@ -434,10 +434,15 @@ public class MapTask extends Task { } statusUpdate(umbilical); collector.flush(); - } finally { - //close - in.close(); // close input + + in.close(); + in = null; + collector.close(); + collector = null; + } finally { + closeQuietly(in); + closeQuietly(collector); } } @@ -753,13 +758,20 @@ public class MapTask extends Task { new WrappedMapper().getMapContext( mapContext); - input.initialize(split, mapperContext); - mapper.run(mapperContext); - mapPhase.complete(); - setPhase(TaskStatus.Phase.SORT); - statusUpdate(umbilical); - input.close(); - output.close(mapperContext); + 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); + } } class DirectMapOutputCollector @@ -1949,4 +1961,55 @@ public class MapTask extends Task { } } + private + void closeQuietly(RecordReader c) { + if (c != null) { + try { + c.close(); + } catch (IOException ie) { + // Ignore + LOG.info("Ignoring exception during close for " + c, ie); + } + } + } + + private + void closeQuietly(MapOutputCollector c) { + if (c != null) { + try { + c.close(); + } catch (Exception ie) { + // Ignore + LOG.info("Ignoring exception during close for " + c, ie); + } + } + } + + private + void closeQuietly( + org.apache.hadoop.mapreduce.RecordReader c) { + if (c != null) { + try { + c.close(); + } catch (Exception ie) { + // Ignore + LOG.info("Ignoring exception during close for " + c, ie); + } + } + } + + private + void closeQuietly( + org.apache.hadoop.mapreduce.RecordWriter c, + org.apache.hadoop.mapreduce.Mapper.Context + mapperContext) { + if (c != null) { + try { + c.close(mapperContext); + } catch (Exception ie) { + // Ignore + LOG.info("Ignoring exception during close for " + c, ie); + } + } + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java index e8d97fab6ef..ea8ef3afdca 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java @@ -37,6 +37,7 @@ 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; @@ -428,14 +429,15 @@ public class ReduceTask extends Task { // make output collector String finalName = getOutputName(getPartition()); - final RecordWriter out = new OldTrackingRecordWriter( + RecordWriter out = new OldTrackingRecordWriter( this, job, reporter, finalName); - + final RecordWriter finalOut = out; + OutputCollector collector = new OutputCollector() { public void collect(OUTKEY key, OUTVALUE value) throws IOException { - out.write(key, value); + finalOut.write(key, value); // indicate that progress update needs to be sent reporter.progress(); } @@ -466,20 +468,14 @@ public class ReduceTask extends Task { values.informReduceProgress(); } - //Clean up: repeated in catch block below reducer.close(); - out.close(reporter); - //End of clean up. - } catch (IOException ioe) { - try { - reducer.close(); - } catch (IOException ignored) {} - - try { - out.close(reporter); - } catch (IOException ignored) {} + reducer = null; - throw ioe; + out.close(reporter); + out = null; + } finally { + IOUtils.cleanup(LOG, reducer); + closeQuietly(out, reporter); } } @@ -645,7 +641,21 @@ public class ReduceTask extends Task { committer, reporter, comparator, keyClass, valueClass); - reducer.run(reducerContext); - trackedRW.close(reducerContext); + try { + reducer.run(reducerContext); + } finally { + trackedRW.close(reducerContext); + } + } + + private + void closeQuietly(RecordWriter c, Reporter r) { + if (c != null) { + try { + c.close(r); + } catch (Exception e) { + LOG.info("Exception in closing " + c, e); + } + } } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReaderWrapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReaderWrapper.java new file mode 100644 index 00000000000..08e7b475174 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReaderWrapper.java @@ -0,0 +1,86 @@ +/** + * 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 + * CombineFileRecordReader. + * + * 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 + implements RecordReader { + private final RecordReader delegate; + + protected CombineFileRecordReaderWrapper(FileInputFormat 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(); + } +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineSequenceFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineSequenceFileInputFormat.java new file mode 100644 index 00000000000..3a5157e15e1 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineSequenceFileInputFormat.java @@ -0,0 +1,66 @@ +/** + * 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 CombineFileInputFormat-equivalent for + * SequenceFileInputFormat. + * + * @see CombineFileInputFormat + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class CombineSequenceFileInputFormat + extends CombineFileInputFormat { + @SuppressWarnings({ "rawtypes", "unchecked" }) + public RecordReader 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 CombineFileRecordReader + * so that it can be used in a CombineFileInputFormat-equivalent + * for SequenceFileInputFormat. + * + * @see CombineFileRecordReader + * @see CombineFileInputFormat + * @see SequenceFileInputFormat + */ + private static class SequenceFileRecordReaderWrapper + extends CombineFileRecordReaderWrapper { + // this constructor signature is required by CombineFileRecordReader + public SequenceFileRecordReaderWrapper(CombineFileSplit split, + Configuration conf, Reporter reporter, Integer idx) throws IOException { + super(new SequenceFileInputFormat(), split, conf, reporter, idx); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineTextInputFormat.java new file mode 100644 index 00000000000..988e6e6160f --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineTextInputFormat.java @@ -0,0 +1,68 @@ +/** + * 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 CombineFileInputFormat-equivalent for + * TextInputFormat. + * + * @see CombineFileInputFormat + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class CombineTextInputFormat + extends CombineFileInputFormat { + @SuppressWarnings({ "rawtypes", "unchecked" }) + public RecordReader 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 CombineFileRecordReader + * so that it can be used in a CombineFileInputFormat-equivalent + * for TextInputFormat. + * + * @see CombineFileRecordReader + * @see CombineFileInputFormat + * @see TextInputFormat + */ + private static class TextRecordReaderWrapper + extends CombineFileRecordReaderWrapper { + // 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); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java index 94345375c52..3a6186b9b91 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java @@ -140,9 +140,12 @@ public class Mapper { */ public void run(Context context) throws IOException, InterruptedException { setup(context); - while (context.nextKeyValue()) { - map(context.getCurrentKey(), context.getCurrentValue(), context); + try { + while (context.nextKeyValue()) { + map(context.getCurrentKey(), context.getCurrentValue(), context); + } + } finally { + cleanup(context); } - cleanup(context); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java index efb0950715e..3df477f13e5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Reducer.java @@ -166,14 +166,17 @@ public class Reducer { */ public void run(Context context) throws IOException, InterruptedException { setup(context); - while (context.nextKey()) { - reduce(context.getCurrentKey(), context.getValues(), context); - // If a back up store is used, reset it - Iterator iter = context.getValues().iterator(); - if(iter instanceof ReduceContext.ValueIterator) { - ((ReduceContext.ValueIterator)iter).resetBackupStore(); + try { + while (context.nextKey()) { + reduce(context.getCurrentKey(), context.getValues(), context); + // If a back up store is used, reset it + Iterator iter = context.getValues().iterator(); + if(iter instanceof ReduceContext.ValueIterator) { + ((ReduceContext.ValueIterator)iter).resetBackupStore(); + } } + } finally { + cleanup(context); } - cleanup(context); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileRecordReaderWrapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileRecordReaderWrapper.java new file mode 100644 index 00000000000..bf847367b6c --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileRecordReaderWrapper.java @@ -0,0 +1,108 @@ +/** + * 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 + * CombineFileRecordReader. + * + * 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 + extends RecordReader { + private final FileSplit fileSplit; + private final RecordReader delegate; + + protected CombineFileRecordReaderWrapper(FileInputFormat 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(); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineSequenceFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineSequenceFileInputFormat.java new file mode 100644 index 00000000000..368254813bd --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineSequenceFileInputFormat.java @@ -0,0 +1,64 @@ +/** + * 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 CombineFileInputFormat-equivalent for + * SequenceFileInputFormat. + * + * @see CombineFileInputFormat + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class CombineSequenceFileInputFormat + extends CombineFileInputFormat { + @SuppressWarnings({ "rawtypes", "unchecked" }) + public RecordReader createRecordReader(InputSplit split, + TaskAttemptContext context) throws IOException { + return new CombineFileRecordReader((CombineFileSplit)split, context, + SequenceFileRecordReaderWrapper.class); + } + + /** + * A record reader that may be passed to CombineFileRecordReader + * so that it can be used in a CombineFileInputFormat-equivalent + * for SequenceFileInputFormat. + * + * @see CombineFileRecordReader + * @see CombineFileInputFormat + * @see SequenceFileInputFormat + */ + private static class SequenceFileRecordReaderWrapper + extends CombineFileRecordReaderWrapper { + // this constructor signature is required by CombineFileRecordReader + public SequenceFileRecordReaderWrapper(CombineFileSplit split, + TaskAttemptContext context, Integer idx) + throws IOException, InterruptedException { + super(new SequenceFileInputFormat(), split, context, idx); + } + } +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineTextInputFormat.java new file mode 100644 index 00000000000..80876112221 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineTextInputFormat.java @@ -0,0 +1,65 @@ +/** + * 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 CombineFileInputFormat-equivalent for + * TextInputFormat. + * + * @see CombineFileInputFormat + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class CombineTextInputFormat + extends CombineFileInputFormat { + public RecordReader createRecordReader(InputSplit split, + TaskAttemptContext context) throws IOException { + return new CombineFileRecordReader( + (CombineFileSplit)split, context, TextRecordReaderWrapper.class); + } + + /** + * A record reader that may be passed to CombineFileRecordReader + * so that it can be used in a CombineFileInputFormat-equivalent + * for TextInputFormat. + * + * @see CombineFileRecordReader + * @see CombineFileInputFormat + * @see TextInputFormat + */ + private static class TextRecordReaderWrapper + extends CombineFileRecordReaderWrapper { + // this constructor signature is required by CombineFileRecordReader + public TextRecordReaderWrapper(CombineFileSplit split, + TaskAttemptContext context, Integer idx) + throws IOException, InterruptedException { + super(new TextInputFormat(), split, context, idx); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java index c33ab38c150..50bfedd6432 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java +++ b/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"); + "N/A", 0, 0, FinalApplicationStatus.UNDEFINED, null, "N/A", 0.0f); } NotRunningJob(ApplicationReport applicationReport, JobState jobState) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java index 8e1be11bfc6..4eaddfcf572 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java +++ b/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"); + "N/A", 0.0f); } 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"); + "N/A", 0.0f); } private ResourceMgrDelegate getRMDelegate() throws YarnRemoteException { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java new file mode 100644 index 00000000000..f76fb22660f --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java @@ -0,0 +1,170 @@ +/** + * 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 format = + new CombineSequenceFileInputFormat(); + 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 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(); + } + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java new file mode 100644 index 00000000000..384bd39c75e --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java @@ -0,0 +1,250 @@ +/** + * 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 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 readSplit(InputFormat format, + InputSplit split, + JobConf job) throws IOException { + List result = new ArrayList(); + RecordReader 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 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 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()); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapperReducerCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapperReducerCleanup.java new file mode 100644 index 00000000000..36ec966b696 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapperReducerCleanup.java @@ -0,0 +1,334 @@ +/** + * 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 { + + /** 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 { + + 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 { + + public void reduce(LongWritable key, Iterable 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 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); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java new file mode 100644 index 00000000000..047f94909f4 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineSequenceFileInputFormat.java @@ -0,0 +1,186 @@ +/** + * 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 format = + new CombineSequenceFileInputFormat(); + for (int i = 0; i < 3; i++) { + int numSplits = + random.nextInt(length/(SequenceFile.SYNC_INTERVAL/20)) + 1; + LOG.info("splitting: requesting = " + numSplits); + List 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 reader = + format.createRecordReader(split, context); + MapContext mcontext = + new MapContextImpl(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(); + } + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java new file mode 100644 index 00000000000..0b44ca610a9 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineTextInputFormat.java @@ -0,0 +1,267 @@ +/** + * 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 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 reader = + format.createRecordReader(split, context); + assertEquals("reader class is CombineFileRecordReader.", + CombineFileRecordReader.class, reader.getClass()); + MapContext mcontext = + new MapContextImpl(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 readSplit(InputFormat format, + InputSplit split, Job job) throws IOException, InterruptedException { + List result = new ArrayList(); + Configuration conf = job.getConfiguration(); + TaskAttemptContext context = MapReduceTestUtil. + createDummyMapTaskAttemptContext(conf); + RecordReader reader = format.createRecordReader(split, + MapReduceTestUtil.createDummyMapTaskAttemptContext(conf)); + MapContext mcontext = + new MapContextImpl(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 splits = format.getSplits(job); + assertEquals("compressed splits == 1", 1, splits.size()); + List 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 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()); + } + } +} diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 536a069809f..1e260185247 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -98,6 +98,10 @@ 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. @@ -177,6 +181,12 @@ 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 @@ -290,6 +300,12 @@ 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 diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java index 8fac8cbe53f..8a824ec8361 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java @@ -29,10 +29,6 @@ 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"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java index db68efde590..26f9978d7a2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java @@ -253,4 +253,16 @@ 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); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java index 69f939c088e..57c334a5bec 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java @@ -213,6 +213,12 @@ 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(); @@ -345,6 +351,12 @@ implements ApplicationReport { builder.setFinalApplicationStatus(convertToProtoFormat(finishState)); } + @Override + public void setProgress(float progress) { + maybeInitBuilder(); + builder.setProgress(progress); + } + @Override public ApplicationReportProto getProto() { mergeLocalToProto(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto index c865c5dda74..a84bf37e39e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto @@ -162,6 +162,7 @@ message ApplicationReportProto { optional ApplicationResourceUsageReportProto app_resource_Usage = 16; optional string originalTrackingUrl = 17; optional ApplicationAttemptIdProto currentApplicationAttemptId = 18; + optional float progress = 19; } enum NodeStateProto { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java index 139a46c1988..2211c480ab9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java @@ -33,13 +33,9 @@ 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; @@ -115,24 +111,7 @@ public class AMRMClientImpl extends AbstractService implements AMRMClient { throw new YarnException(e); } - if (UserGroupInformation.isSecurityEnabled()) { - String tokenURLEncodedStr = System.getenv().get( - ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME); - Token token = new Token(); - - 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); - } - + // CurrentUser should already have AMToken loaded. rmClient = currentUser.doAs(new PrivilegedAction() { @Override public AMRMProtocol run() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java index d884e769fa4..5797e341dab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java @@ -20,6 +20,7 @@ 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; @@ -34,7 +35,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%35s" + + "%30s\t%20s\t%10s\t%10s\t%18s\t%18s\t%15s\t%35s" + System.getProperty("line.separator"); public static void main(String[] args) throws Exception { @@ -98,12 +99,15 @@ public class ApplicationCLI extends YarnCLI { writer.println("Total Applications:" + appsReport.size()); writer.printf(APPLICATIONS_PATTERN, "Application-Id", "Application-Name", "User", "Queue", "State", "Final-State", - "Tracking-URL"); + "Progress", "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(), appReport.getOriginalTrackingUrl()); + .getFinalApplicationStatus(), + progress, appReport.getOriginalTrackingUrl()); } writer.flush(); } @@ -147,6 +151,10 @@ 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 : "); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java index fbe4809114b..716cf2562f2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java +++ b/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"); + FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f); when(client.getApplicationReport(any(ApplicationId.class))).thenReturn( newApplicationReport); int result = cli.run(new String[] { "-status", applicationId.toString() }); @@ -91,6 +91,7 @@ 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"); @@ -111,7 +112,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"); + FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f); List applicationReports = new ArrayList(); applicationReports.add(newApplicationReport); when(client.getApplicationList()).thenReturn(applicationReports); @@ -124,10 +125,12 @@ 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.println("Final-State\t Tracking-URL"); + pw.print("Final-State\t Progress"); + pw.println("\t Tracking-URL"); pw.print(" application_1234_0005\t "); pw.print("appname\t user\t queue\t FINISHED\t "); - pw.println("SUCCEEDED\t N/A"); + pw.print("SUCCEEDED\t 53.79%"); + pw.println("\t N/A"); pw.close(); String appsReportStr = baos.toString("UTF-8"); Assert.assertEquals(appsReportStr, sysOutStream.toString()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java index 8ce9808f276..7dc25de8208 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java @@ -333,7 +333,8 @@ public class BuilderUtils { ClientToken clientToken, YarnApplicationState state, String diagnostics, String url, long startTime, long finishTime, FinalApplicationStatus finalStatus, - ApplicationResourceUsageReport appResources, String origTrackingUrl) { + ApplicationResourceUsageReport appResources, String origTrackingUrl, + float progress) { ApplicationReport report = recordFactory .newRecordInstance(ApplicationReport.class); report.setApplicationId(applicationId); @@ -352,6 +353,7 @@ public class BuilderUtils { report.setFinalApplicationStatus(finalStatus); report.setApplicationResourceUsageReport(appResources); report.setOriginalTrackingUrl(origTrackingUrl); + report.setProgress(progress); return report; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java index d82771b8415..aa211a62f96 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/WebServicesTestUtils.java @@ -85,5 +85,10 @@ 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)); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java index 89441d74d25..08768337ed4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java +++ b/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.checkStringMatch("hadoopBuildVersion", + WebServicesTestUtils.checkStringEqual("hadoopBuildVersion", VersionInfo.getBuildVersion(), hadoopBuildVersion); WebServicesTestUtils.checkStringMatch("hadoopVersion", VersionInfo.getVersion(), hadoopVersion); WebServicesTestUtils.checkStringMatch("resourceManagerVersionBuiltOn", YarnVersionInfo.getDate(), resourceManagerVersionBuiltOn); - WebServicesTestUtils.checkStringMatch("resourceManagerBuildVersion", + WebServicesTestUtils.checkStringEqual("resourceManagerBuildVersion", YarnVersionInfo.getBuildVersion(), resourceManagerBuildVersion); WebServicesTestUtils.checkStringMatch("resourceManagerVersion", YarnVersionInfo.getVersion(), resourceManagerVersion); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java index fb95550eb14..a9d40ebad13 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java +++ b/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 token = + Token appMasterToken = new Token(id, this.rmContext.getApplicationTokenSecretManager()); InetSocketAddress serviceAddr = conf.getSocketAddr( @@ -212,16 +212,11 @@ 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 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); + // but this token is directly provided to the AMs + SecurityUtil.setTokenService(appMasterToken, serviceAddr); - // Add the RM token - credentials.addToken(token.getService(), token); + // Add the ApplicationMaster token + credentials.addToken(appMasterToken.getService(), appMasterToken); DataOutputBuffer dob = new DataOutputBuffer(); credentials.writeTokenStorageToStream(dob); container.setContainerTokens( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java index 5c94f927838..58cd676b235 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java +++ b/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,4 +208,14 @@ 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())); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java index 7b63cbe8048..874232bb165 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java +++ b/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,6 +437,7 @@ 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(); @@ -446,8 +447,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(); } @@ -462,7 +463,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); + appUsageReport, origTrackingUrl, progress); } finally { this.readLock.unlock(); } diff --git a/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 b/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 index fa01a0bb993..a8e71735efc 100644 --- a/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 +++ b/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,6 +35,7 @@ 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; @@ -45,6 +46,9 @@ 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); @@ -180,15 +184,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(FSSchedulerApp application, Priority priority, - FSSchedulerNode node, Container container, boolean alreadyReserved) { + private void reserve(Priority priority, FSSchedulerNode node, + Container container, boolean alreadyReserved) { LOG.info("Making reservation: node=" + node.getHostName() + " app_id=" + app.getApplicationId()); if (!alreadyReserved) { - getMetrics().reserveResource(application.getUser(), container.getResource()); - RMContainer rmContainer = application.reserve(node, priority, null, + getMetrics().reserveResource(app.getUser(), container.getResource()); + RMContainer rmContainer = app.reserve(node, priority, null, container); - node.reserveResource(application, priority, rmContainer); + node.reserveResource(app, priority, rmContainer); getMetrics().reserveResource(app.getUser(), container.getResource()); scheduler.getRootQueueMetrics().reserveResource(app.getUser(), @@ -197,25 +201,24 @@ public class AppSchedulable extends Schedulable { else { RMContainer rmContainer = node.getReservedContainer(); - application.reserve(node, priority, rmContainer, container); - node.reserveResource(application, priority, rmContainer); + app.reserve(node, priority, rmContainer, container); + node.reserveResource(app, priority, rmContainer); } } /** - * Remove the reservation on {@code node} for {@ application} at the given + * Remove the reservation on {@code node} at the given * {@link Priority}. This dispatches to the SchedulerApp and SchedulerNode * handlers for an unreservation. */ - private void unreserve(FSSchedulerApp application, Priority priority, - FSSchedulerNode node) { + public void unreserve(Priority priority, FSSchedulerNode node) { RMContainer rmContainer = node.getReservedContainer(); - application.unreserve(node, priority); - node.unreserveResource(application); + app.unreserve(node, priority); + node.unreserveResource(app); getMetrics().unreserveResource( - application.getUser(), rmContainer.getContainer().getResource()); + app.getUser(), rmContainer.getContainer().getResource()); scheduler.getRootQueueMetrics().unreserveResource( - application.getUser(), rmContainer.getContainer().getResource()); + app.getUser(), rmContainer.getContainer().getResource()); } /** @@ -224,8 +227,8 @@ public class AppSchedulable extends Schedulable { * sure the particular request should be facilitated by this node. */ private Resource assignContainer(FSSchedulerNode node, - FSSchedulerApp application, Priority priority, - ResourceRequest request, NodeType type, boolean reserved) { + Priority priority, ResourceRequest request, NodeType type, + boolean reserved) { // How much does this request need? Resource capability = request.getCapability(); @@ -237,7 +240,7 @@ public class AppSchedulable extends Schedulable { if (reserved) { container = node.getReservedContainer().getContainer(); } else { - container = createContainer(application, node, capability, priority); + container = createContainer(app, node, capability, priority); } // Can we allocate a container on this node? @@ -247,9 +250,12 @@ public class AppSchedulable extends Schedulable { if (availableContainers > 0) { // Inform the application of the new container for this request RMContainer allocatedContainer = - application.allocate(type, node, priority, request, container); + app.allocate(type, node, priority, request, container); if (allocatedContainer == null) { // Did the application need this resource? + if (reserved) { + unreserve(priority, node); + } return Resources.none(); } else { @@ -262,17 +268,17 @@ public class AppSchedulable extends Schedulable { // If we had previously made a reservation, delete it if (reserved) { - unreserve(application, priority, node); + unreserve(priority, node); } // Inform the node - node.allocateContainer(application.getApplicationId(), + node.allocateContainer(app.getApplicationId(), allocatedContainer); return container.getResource(); } else { // The desired container won't fit here, so reserve - reserve(application, priority, node, container, reserved); + reserve(priority, node, container, reserved); return FairScheduler.CONTAINER_RESERVED; } @@ -287,7 +293,7 @@ public class AppSchedulable extends Schedulable { // Make sure the application still needs requests at this priority if (app.getTotalRequiredResources(priority) == 0) { - unreserve(app, priority, node); + unreserve(priority, node); return Resources.none(); } } else { @@ -304,7 +310,8 @@ public class AppSchedulable extends Schedulable { // (not scheduled) in order to promote better locality. synchronized (app) { for (Priority priority : prioritiesToTry) { - if (app.getTotalRequiredResources(priority) <= 0) { + if (app.getTotalRequiredResources(priority) <= 0 || + !hasContainerForNode(priority, node)) { continue; } @@ -321,14 +328,14 @@ public class AppSchedulable extends Schedulable { if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0 && localRequest != null && localRequest.getNumContainers() != 0) { - return assignContainer(node, app, priority, + return assignContainer(node, 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, app, priority, rackLocalRequest, + return assignContainer(node, priority, rackLocalRequest, NodeType.RACK_LOCAL, reserved); } @@ -336,7 +343,7 @@ public class AppSchedulable extends Schedulable { ResourceRequest.ANY); if (offSwitchRequest != null && offSwitchRequest.getNumContainers() != 0 && allowedLocality.equals(NodeType.OFF_SWITCH)) { - return assignContainer(node, app, priority, offSwitchRequest, + return assignContainer(node, priority, offSwitchRequest, NodeType.OFF_SWITCH, reserved); } } @@ -352,4 +359,16 @@ 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()); + } } \ No newline at end of file diff --git a/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 b/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 index ff5344d12a4..09b14a4b98e 100644 --- a/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 +++ b/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,6 +33,7 @@ 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( @@ -126,8 +127,8 @@ public class FSLeafQueue extends FSQueue { + demand); } demand = Resources.add(demand, toAdd); - if (Resources.greaterThanOrEqual(demand, maxRes)) { - demand = maxRes; + demand = Resources.componentwiseMin(demand, maxRes); + if (Resources.equals(demand, maxRes)) { break; } } @@ -153,7 +154,7 @@ public class FSLeafQueue extends FSQueue { for (AppSchedulable sched : appScheds) { if (sched.getRunnable()) { assigned = sched.assignContainer(node); - if (Resources.greaterThan(assigned, Resources.none())) { + if (!assigned.equals(Resources.none())) { break; } } diff --git a/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 b/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 index 298ceeed056..253052a4181 100644 --- a/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 +++ b/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,6 +29,7 @@ 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( @@ -87,8 +88,8 @@ public class FSParentQueue extends FSQueue { " now " + demand); } demand = Resources.add(demand, toAdd); - if (Resources.greaterThanOrEqual(demand, maxRes)) { - demand = maxRes; + demand = Resources.componentwiseMin(demand, maxRes); + if (Resources.equals(demand, maxRes)) { break; } } @@ -135,16 +136,14 @@ public class FSParentQueue extends FSQueue { Resource assigned = Resources.none(); // If this queue is over its limit, reject - if (Resources.greaterThan(getResourceUsage(), - queueMgr.getMaxResources(getName()))) { + if (!assignContainerPreCheck(node)) { return assigned; } Collections.sort(childQueues, policy.getComparator()); for (FSQueue child : childQueues) { assigned = child.assignContainer(node); - if (node.getReservedContainer() != null - || Resources.greaterThan(assigned, Resources.none())) { + if (!Resources.equals(assigned, Resources.none())) { break; } } diff --git a/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 b/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 index 0a03749d326..f4b29d30b0a 100644 --- a/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 +++ b/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,6 +32,7 @@ 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; @@ -161,7 +162,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.greaterThan(getResourceUsage(), + if (!Resources.fitsIn(getResourceUsage(), queueMgr.getMaxResources(getName())) || node.getReservedContainer() != null) { return false; diff --git a/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 b/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 index 7cde2fb6827..533d5c3ddbe 100644 --- a/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 +++ b/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 = recordFactory.newRecordInstance(Resource.class); + private Resource availableResource; 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.setMemory(node.getTotalCapability().getMemory()); + this.availableResource = Resources.clone(node.getTotalCapability()); } public RMNode getRMNode() { diff --git a/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 b/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 index df575860afd..15f3eba65e8 100644 --- a/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 +++ b/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,6 +41,7 @@ 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; @@ -52,6 +53,9 @@ 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; @@ -111,6 +115,9 @@ 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); @@ -246,8 +253,10 @@ 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(sched.getMinShare(), sched.getDemand()); - return Resources.lessThan(sched.getResourceUsage(), desiredShare); + Resource desiredShare = Resources.min(RESOURCE_CALCULATOR, clusterCapacity, + sched.getMinShare(), sched.getDemand()); + return Resources.lessThan(RESOURCE_CALCULATOR, clusterCapacity, + sched.getResourceUsage(), desiredShare); } /** @@ -255,9 +264,10 @@ public class FairScheduler implements ResourceScheduler { * defined as being below half its fair share. */ boolean isStarvedForFairShare(FSLeafQueue sched) { - Resource desiredFairShare = Resources.max( + Resource desiredFairShare = Resources.max(RESOURCE_CALCULATOR, clusterCapacity, Resources.multiply(sched.getFairShare(), .5), sched.getDemand()); - return Resources.lessThan(sched.getResourceUsage(), desiredFairShare); + return Resources.lessThan(RESOURCE_CALCULATOR, clusterCapacity, + sched.getResourceUsage(), desiredFairShare); } /** @@ -283,7 +293,8 @@ public class FairScheduler implements ResourceScheduler { for (FSLeafQueue sched : queueMgr.getLeafQueues()) { resToPreempt = Resources.add(resToPreempt, resToPreempt(sched, curTime)); } - if (Resources.greaterThan(resToPreempt, Resources.none())) { + if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity, resToPreempt, + Resources.none())) { preemptResources(queueMgr.getLeafQueues(), resToPreempt); } } @@ -309,7 +320,8 @@ public class FairScheduler implements ResourceScheduler { // Collect running containers from over-scheduled queues List runningContainers = new ArrayList(); for (FSLeafQueue sched : scheds) { - if (Resources.greaterThan(sched.getResourceUsage(), sched.getFairShare())) { + if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity, + sched.getResourceUsage(), sched.getFairShare())) { for (AppSchedulable as : sched.getAppSchedulables()) { for (RMContainer c : as.getApp().getLiveContainers()) { runningContainers.add(c); @@ -332,7 +344,8 @@ 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(sched.getResourceUsage(), sched.getFairShare())) { + if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity, + sched.getResourceUsage(), sched.getFairShare())) { LOG.info("Preempting container (prio=" + container.getContainer().getPriority() + "res=" + container.getContainer().getResource() + ") from queue " + sched.getName()); @@ -345,7 +358,8 @@ public class FairScheduler implements ResourceScheduler { toPreempt = Resources.subtract(toPreempt, container.getContainer().getResource()); - if (Resources.equals(toPreempt, Resources.none())) { + if (Resources.lessThanOrEqual(RESOURCE_CALCULATOR, clusterCapacity, + toPreempt, Resources.none())) { break; } } @@ -369,17 +383,21 @@ public class FairScheduler implements ResourceScheduler { Resource resDueToMinShare = Resources.none(); Resource resDueToFairShare = Resources.none(); if (curTime - sched.getLastTimeAtMinShare() > minShareTimeout) { - Resource target = Resources.min(sched.getMinShare(), sched.getDemand()); - resDueToMinShare = Resources.max(Resources.none(), - Resources.subtract(target, sched.getResourceUsage())); + Resource target = Resources.min(RESOURCE_CALCULATOR, clusterCapacity, + sched.getMinShare(), sched.getDemand()); + resDueToMinShare = Resources.max(RESOURCE_CALCULATOR, clusterCapacity, + Resources.none(), Resources.subtract(target, sched.getResourceUsage())); } if (curTime - sched.getLastTimeAtHalfFairShare() > fairShareTimeout) { - Resource target = Resources.min(sched.getFairShare(), sched.getDemand()); - resDueToFairShare = Resources.max(Resources.none(), - Resources.subtract(target, sched.getResourceUsage())); + 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(resDueToMinShare, resDueToFairShare); - if (Resources.greaterThan(resToPreempt, Resources.none())) { + Resource resToPreempt = Resources.max(RESOURCE_CALCULATOR, clusterCapacity, + resDueToMinShare, resDueToFairShare); + if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity, + resToPreempt, Resources.none())) { String message = "Should preempt " + resToPreempt + " res for queue " + sched.getName() + ": resDueToMinShare = " + resDueToMinShare + ", resDueToFairShare = " + resDueToFairShare; @@ -788,21 +806,32 @@ public class FairScheduler implements ResourceScheduler { AppSchedulable reservedAppSchedulable = node.getReservedAppSchedulable(); if (reservedAppSchedulable != null) { - // Reservation exists; try to fulfill the reservation - LOG.info("Trying to fulfill reservation for application " - + reservedAppSchedulable.getApp().getApplicationAttemptId() - + " on node: " + nm); + 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); + node.getReservedAppSchedulable().assignReservedContainer(node); + } } - else { + if (reservedAppSchedulable == null) { // No reservation, schedule at queue which is farthest below fair share int assignedContainers = 0; while (node.getReservedContainer() == null) { boolean assignedContainer = false; - if (Resources.greaterThan( - queueMgr.getRootQueue().assignContainer(node), - Resources.none())) { + if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity, + queueMgr.getRootQueue().assignContainer(node), + Resources.none())) { assignedContainer = true; } if (!assignedContainer) { break; } diff --git a/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 b/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 index e4efb937e27..1365565e563 100644 --- a/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 +++ b/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,6 +42,7 @@ 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; @@ -474,8 +475,8 @@ public class QueueManager { } queueAcls.put(queueName, acls); if (maxQueueResources.containsKey(queueName) && minQueueResources.containsKey(queueName) - && Resources.lessThan(maxQueueResources.get(queueName), - minQueueResources.get(queueName))) { + && !Resources.fitsIn(minQueueResources.get(queueName), + maxQueueResources.get(queueName))) { LOG.warn(String.format("Queue %s has max resources %d less than min resources %d", queueName, maxQueueResources.get(queueName), minQueueResources.get(queueName))); } @@ -504,7 +505,7 @@ public class QueueManager { if (maxQueueResource != null) { return maxQueueResource; } else { - return Resources.createResource(Integer.MAX_VALUE); + return Resources.createResource(Integer.MAX_VALUE, Integer.MAX_VALUE); } } diff --git a/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 b/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 deleted file mode 100644 index 1e700340b9b..00000000000 --- a/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 +++ /dev/null @@ -1,150 +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.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; - }} diff --git a/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 b/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 index 4e229315cc3..e94d03217a3 100644 --- a/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 +++ b/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,7 +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.scheduler.fair.Resources; +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.Schedulable; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy; @@ -31,6 +32,8 @@ 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 @@ -59,15 +62,19 @@ public class FairSharePolicy extends SchedulingPolicy { public int compare(Schedulable s1, Schedulable s2) { double minShareRatio1, minShareRatio2; double useToWeightRatio1, useToWeightRatio2; - 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 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 one = Resources.createResource(1); minShareRatio1 = (double) s1.getResourceUsage().getMemory() - / Resources.max(minShare1, one).getMemory(); + / Resources.max(RESOURCE_CALCULATOR, null, minShare1, one).getMemory(); minShareRatio2 = (double) s2.getResourceUsage().getMemory() - / Resources.max(minShare2, one).getMemory(); + / Resources.max(RESOURCE_CALCULATOR, null, minShare2, one).getMemory(); useToWeightRatio1 = s1.getResourceUsage().getMemory() / s1.getWeight(); useToWeightRatio2 = s2.getResourceUsage().getMemory() / s2.getWeight(); int res = 0; @@ -161,9 +168,11 @@ public class FairSharePolicy extends SchedulingPolicy { for (Schedulable sched : schedulables) { Resources.addTo(totalDemand, sched.getDemand()); } - Resource cap = Resources.min(totalDemand, totalResources); + Resource cap = Resources.min(RESOURCE_CALCULATOR, null, totalDemand, + totalResources); double rMax = 1.0; - while (Resources.lessThan(resUsedWithWeightToResRatio(rMax, schedulables), + while (Resources.lessThan(RESOURCE_CALCULATOR, null, + resUsedWithWeightToResRatio(rMax, schedulables), cap)) { rMax *= 2.0; } @@ -172,7 +181,8 @@ 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(resUsedWithWeightToResRatio(mid, schedulables), + if (Resources.lessThan(RESOURCE_CALCULATOR, null, + resUsedWithWeightToResRatio(mid, schedulables), cap)) { left = mid; } else { diff --git a/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 b/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 index 25766ea6222..c8fabee09ed 100644 --- a/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 +++ b/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; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java index 32505012115..bfa20927b2e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java @@ -20,6 +20,7 @@ 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; @@ -28,11 +29,10 @@ 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,9 +59,17 @@ 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 Map amContainerEnv; + public ByteBuffer amTokens; public MyContainerManager() { } @@ -70,7 +78,7 @@ public class TestAMAuthorization { public StartContainerResponse startContainer(StartContainerRequest request) throws YarnRemoteException { - amContainerEnv = request.getContainerLaunchContext().getEnvironment(); + amTokens = request.getContainerLaunchContext().getContainerTokens(); return null; } @@ -93,9 +101,6 @@ public class TestAMAuthorization { public MockRMWithAMS(Configuration conf, ContainerManager containerManager) { super(conf, containerManager); - conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, - "kerberos"); - UserGroupInformation.setConfiguration(conf); } @Override @@ -105,7 +110,6 @@ public class TestAMAuthorization { @Override protected ApplicationMasterService createApplicationMasterService() { - return new ApplicationMasterService(getRMContext(), this.scheduler); } } @@ -113,7 +117,8 @@ public class TestAMAuthorization { @Test public void testAuthorizedAccess() throws Exception { MyContainerManager containerManager = new MyContainerManager(); - final MockRM rm = new MockRMWithAMS(new Configuration(), containerManager); + final MockRM rm = + new MockRMWithAMS(confWithSecurityEnabled, containerManager); rm.start(); MockNM nm1 = rm.registerNode("localhost:1234", 5120); @@ -126,11 +131,11 @@ public class TestAMAuthorization { nm1.nodeHeartbeat(true); int waitCount = 0; - while (containerManager.amContainerEnv == null && waitCount++ < 20) { + while (containerManager.amTokens == null && waitCount++ < 20) { LOG.info("Waiting for AM Launch to happen.."); Thread.sleep(1000); } - Assert.assertNotNull(containerManager.amContainerEnv); + Assert.assertNotNull(containerManager.amTokens); RMAppAttempt attempt = app.getCurrentAppAttempt(); ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId(); @@ -142,12 +147,12 @@ 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 token = new Token(); - token.decodeFromUrlString(tokenURLEncodedStr); - currentUser.addToken(token); + Credentials credentials = new Credentials(); + DataInputByteBuffer buf = new DataInputByteBuffer(); + containerManager.amTokens.rewind(); + buf.reset(containerManager.amTokens); + credentials.readTokenStorageStream(buf); + currentUser.addCredentials(credentials); AMRMProtocol client = currentUser .doAs(new PrivilegedAction() { @@ -172,7 +177,7 @@ public class TestAMAuthorization { @Test public void testUnauthorizedAccess() throws Exception { MyContainerManager containerManager = new MyContainerManager(); - MockRM rm = new MockRMWithAMS(new Configuration(), containerManager); + MockRM rm = new MockRMWithAMS(confWithSecurityEnabled, containerManager); rm.start(); MockNM nm1 = rm.registerNode("localhost:1234", 5120); @@ -182,17 +187,16 @@ public class TestAMAuthorization { nm1.nodeHeartbeat(true); int waitCount = 0; - while (containerManager.amContainerEnv == null && waitCount++ < 20) { + while (containerManager.amTokens == null && waitCount++ < 20) { LOG.info("Waiting for AM Launch to happen.."); Thread.sleep(1000); } - Assert.assertNotNull(containerManager.amContainerEnv); + Assert.assertNotNull(containerManager.amTokens); 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( @@ -202,13 +206,8 @@ 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 token = new Token(); - token.decodeFromUrlString(tokenURLEncodedStr); - currentUser.addToken(token); + // First try contacting NM without tokens AMRMProtocol client = currentUser .doAs(new PrivilegedAction() { @Override @@ -217,9 +216,39 @@ 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() { + @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); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResources.java new file mode 100644 index 00000000000..f4e70f7a1f7 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResources.java @@ -0,0 +1,43 @@ +/** + * 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))); + } +} diff --git a/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 b/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 index 039e0b0b6da..e6761b9c601 100644 --- a/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 +++ b/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,4 +1520,28 @@ 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()); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java index 412ba3e54df..af5ff50e2ea 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestApplicationTokens.java +++ b/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,6 +51,14 @@ 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. @@ -61,7 +69,8 @@ public class TestApplicationTokens { public void testTokenExpiry() throws Exception { MyContainerManager containerManager = new MyContainerManager(); - final MockRM rm = new MockRMWithAMS(new Configuration(), containerManager); + final MockRM rm = + new MockRMWithAMS(confWithSecurityEnabled, containerManager); rm.start(); final Configuration conf = rm.getConfig(); @@ -76,11 +85,11 @@ public class TestApplicationTokens { nm1.nodeHeartbeat(true); int waitCount = 0; - while (containerManager.amContainerEnv == null && waitCount++ < 20) { + while (containerManager.amTokens == null && waitCount++ < 20) { LOG.info("Waiting for AM Launch to happen.."); Thread.sleep(1000); } - Assert.assertNotNull(containerManager.amContainerEnv); + Assert.assertNotNull(containerManager.amTokens); RMAppAttempt attempt = app.getCurrentAppAttempt(); ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId(); @@ -89,13 +98,12 @@ public class TestApplicationTokens { UserGroupInformation currentUser = UserGroupInformation .createRemoteUser(applicationAttemptId.toString()); - String tokenURLEncodedStr = - containerManager.amContainerEnv - .get(ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME); - LOG.info("AppMasterToken is " + tokenURLEncodedStr); - Token token = new Token(); - token.decodeFromUrlString(tokenURLEncodedStr); - currentUser.addToken(token); + Credentials credentials = new Credentials(); + DataInputByteBuffer buf = new DataInputByteBuffer(); + containerManager.amTokens.rewind(); + buf.reset(containerManager.amTokens); + credentials.readTokenStorageStream(buf); + currentUser.addCredentials(credentials); rmClient = createRMClient(rm, conf, rpc, currentUser); @@ -152,9 +160,9 @@ public class TestApplicationTokens { @Test public void testMasterKeyRollOver() throws Exception { - Configuration config = new Configuration(); MyContainerManager containerManager = new MyContainerManager(); - final MockRM rm = new MockRMWithAMS(config, containerManager); + final MockRM rm = + new MockRMWithAMS(confWithSecurityEnabled, containerManager); rm.start(); final Configuration conf = rm.getConfig(); @@ -169,11 +177,11 @@ public class TestApplicationTokens { nm1.nodeHeartbeat(true); int waitCount = 0; - while (containerManager.amContainerEnv == null && waitCount++ < 20) { + while (containerManager.amTokens == null && waitCount++ < 20) { LOG.info("Waiting for AM Launch to happen.."); Thread.sleep(1000); } - Assert.assertNotNull(containerManager.amContainerEnv); + Assert.assertNotNull(containerManager.amTokens); RMAppAttempt attempt = app.getCurrentAppAttempt(); ApplicationAttemptId applicationAttemptId = attempt.getAppAttemptId(); @@ -182,13 +190,12 @@ public class TestApplicationTokens { UserGroupInformation currentUser = UserGroupInformation .createRemoteUser(applicationAttemptId.toString()); - String tokenURLEncodedStr = - containerManager.amContainerEnv - .get(ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME); - LOG.info("AppMasterToken is " + tokenURLEncodedStr); - Token token = new Token(); - token.decodeFromUrlString(tokenURLEncodedStr); - currentUser.addToken(token); + Credentials credentials = new Credentials(); + DataInputByteBuffer buf = new DataInputByteBuffer(); + containerManager.amTokens.rewind(); + buf.reset(containerManager.amTokens); + credentials.readTokenStorageStream(buf); + currentUser.addCredentials(credentials); rmClient = createRMClient(rm, conf, rpc, currentUser); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java index c410478596b..f07becb467d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java +++ b/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.checkStringMatch("hadoopBuildVersion", + WebServicesTestUtils.checkStringEqual("hadoopBuildVersion", VersionInfo.getBuildVersion(), hadoopBuildVersion); WebServicesTestUtils.checkStringMatch("hadoopVersion", VersionInfo.getVersion(), hadoopVersion); WebServicesTestUtils.checkStringMatch("resourceManagerVersionBuiltOn", YarnVersionInfo.getDate(), resourceManagerVersionBuiltOn); - WebServicesTestUtils.checkStringMatch("resourceManagerBuildVersion", + WebServicesTestUtils.checkStringEqual("resourceManagerBuildVersion", YarnVersionInfo.getBuildVersion(), resourceManagerBuildVersion); WebServicesTestUtils.checkStringMatch("resourceManagerVersion", YarnVersionInfo.getVersion(), resourceManagerVersion);