You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2013/04/26 02:55:48 UTC

svn commit: r1476006 - in /hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/native/ src/main/webapps/datanode/ src/main/webapps/hdfs/ src/main/webapps/sec...

Author: szetszwo
Date: Fri Apr 26 00:55:45 2013
New Revision: 1476006

URL: http://svn.apache.org/r1476006
Log:
Merge r1471229 through r1476005 from trunk.

Modified:
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/   (props changed)
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/   (props changed)
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/native/   (props changed)
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/   (props changed)
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/   (props changed)
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/   (props changed)
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs/   (props changed)
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs:r1471229-1476005

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1476006&r1=1476005&r2=1476006&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Fri Apr 26 00:55:45 2013
@@ -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)
 

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java:r1471229-1476005

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

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java?rev=1476006&r1=1476005&r2=1476006&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java Fri Apr 26 00:55:45 2013
@@ -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);
     }
     

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/native/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native:r1471229-1476005

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode:r1471229-1476005

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs:r1471229-1476005

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary:r1471229-1476005

Propchange: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs:r1471229-1476005

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java?rev=1476006&r1=1476005&r2=1476006&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java Fri Apr 26 00:55:45 2013
@@ -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;
+  }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java?rev=1476006&r1=1476005&r2=1476006&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java Fri Apr 26 00:55:45 2013
@@ -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);
+  }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java?rev=1476006&r1=1476005&r2=1476006&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java Fri Apr 26 00:55:45 2013
@@ -217,11 +217,12 @@ public class TestNNStorageRetentionManag
     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);
   }