|
@@ -8,9 +8,6 @@ Trunk (Unreleased)
|
|
|
|
|
|
HDFS-3034. Remove the deprecated DFSOutputStream.sync() method. (szetszwo)
|
|
|
|
|
|
- HDFS-2802. Add HDFS Snapshot feature. (See breakdown of tasks below for
|
|
|
- subtasks and contributors)
|
|
|
-
|
|
|
NEW FEATURES
|
|
|
|
|
|
HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
|
|
@@ -265,782 +262,785 @@ Trunk (Unreleased)
|
|
|
HDFS-4687. TestDelegationTokenForProxyUser#testWebHdfsDoAs is flaky with
|
|
|
JDK7. (Andrew Wang via atm)
|
|
|
|
|
|
- BREAKDOWN OF HDFS-2802 HDFS SNAPSHOT SUBTASKS AND RELATED JIRAS
|
|
|
+Release 2.0.5-beta - UNRELEASED
|
|
|
|
|
|
- HDFS-4076. Support snapshot of single files. (szetszwo)
|
|
|
+ INCOMPATIBLE CHANGES
|
|
|
|
|
|
- HDFS-4082. Add editlog opcodes for snapshot create and delete operations.
|
|
|
- (suresh via szetszwo)
|
|
|
+ HDFS-4305. Add a configurable limit on number of blocks per file, and min
|
|
|
+ block size. (Andrew Wang via atm)
|
|
|
|
|
|
- HDFS-4086. Add editlog opcodes to allow and disallow snapshots on a
|
|
|
- directory. (Brandon Li via suresh)
|
|
|
+ HDFS-4434. Provide a mapping from INodeId to INode. (suresh)
|
|
|
|
|
|
- HDFS-4083. Protocol changes for snapshots. (suresh)
|
|
|
+ HDFS-2802. Add HDFS Snapshot feature. (See breakdown of tasks below for
|
|
|
+ subtasks and contributors)
|
|
|
|
|
|
- HDFS-4077. Add support for Snapshottable Directory. (szetszwo via suresh)
|
|
|
+ NEW FEATURES
|
|
|
|
|
|
- HDFS-4087. Protocol changes for listSnapshots functionality.
|
|
|
- (Brandon Li via suresh)
|
|
|
+ HDFS-1804. Add a new block-volume device choosing policy that looks at
|
|
|
+ free space. (atm)
|
|
|
|
|
|
- HDFS-4079. Add SnapshotManager which maintains a list for all the
|
|
|
- snapshottable directories and supports snapshot methods such as setting a
|
|
|
- directory to snapshottable and creating a snapshot. (szetszwo)
|
|
|
+ HDFS-4296. Reserve layout version for release 1.2.0. (suresh)
|
|
|
|
|
|
- HDFS-4078. Handle replication in snapshots. (szetszwo)
|
|
|
+ HDFS-4334. Add a unique id to INode. (Brandon Li via szetszwo)
|
|
|
|
|
|
- HDFS-4084. Provide CLI support to allow and disallow snapshot
|
|
|
- on a directory. (Brondon Li via suresh)
|
|
|
+ HDFS-4339. Persist inode id in fsimage and editlog. (Brandon Li via
|
|
|
+ suresh)
|
|
|
|
|
|
- HDFS-4091. Add snapshot quota to limit the number of snapshots allowed.
|
|
|
- (szetszwo)
|
|
|
+ HDFS-4340. Update addBlock() to inculde inode id as additional argument.
|
|
|
+ (Brandon Li via suresh)
|
|
|
|
|
|
- HDFS-4097. Provide CLI support for createSnapshot. (Brandon Li via suresh)
|
|
|
+ HDFS-4502. JsonUtil.toFileStatus(..) should check if the fileId property
|
|
|
+ exists. (Brandon Li via suresh)
|
|
|
|
|
|
- HDFS-4092. Update file deletion logic for snapshot so that the current inode
|
|
|
- is removed from the circular linked list; and if some blocks at the end of
|
|
|
- the block list no longer belong to any other inode, collect them and update
|
|
|
- the block list. (szetszwo)
|
|
|
+ HDFS-2576. Enhances the DistributedFileSystem's create API so that clients
|
|
|
+ can specify favored datanodes for a file's blocks. (ddas)
|
|
|
|
|
|
- HDFS-4111. Support snapshot of subtrees. (szetszwo via suresh)
|
|
|
+ HDFS-347. DFS read performance suboptimal when client co-located on nodes
|
|
|
+ with data. (Colin Patrick McCabe via todd and atm)
|
|
|
|
|
|
- HDFS-4119. Complete the allowSnapshot code and add a test for it. (szetszwo)
|
|
|
+ HADOOP-8562. Enhancements to support Hadoop on Windows Server and Windows
|
|
|
+ Azure environments. (See breakdown of tasks below for subtasks and
|
|
|
+ contributors)
|
|
|
+
|
|
|
+ IMPROVEMENTS
|
|
|
|
|
|
- HDFS-4133. Add testcases for testing basic snapshot functionalities.
|
|
|
- (Jing Zhao via suresh)
|
|
|
+ HDFS-4222. NN is unresponsive and loses heartbeats from DNs when
|
|
|
+ configured to use LDAP and LDAP has issues. (Xiaobo Peng, suresh)
|
|
|
|
|
|
- HDFS-4116. Add auditlog for some snapshot operations. (Jing Zhao via suresh)
|
|
|
+ HDFS-4304. Make FSEditLogOp.MAX_OP_SIZE configurable. (Colin Patrick
|
|
|
+ McCabe via atm)
|
|
|
|
|
|
- HDFS-4095. Add some snapshot related metrics. (Jing Zhao via suresh)
|
|
|
+ HDFS-4518. Finer grained metrics for HDFS capacity.
|
|
|
+ (Arpit Agarwal via suresh)
|
|
|
|
|
|
- HDFS-4141. Support directory diff - the difference between the current state
|
|
|
- and a previous snapshot of an INodeDirectory. (szetszwo)
|
|
|
+ HDFS-4519. Support overriding jsvc binary and log file locations
|
|
|
+ when launching secure datanode. (Chris Nauroth via suresh)
|
|
|
|
|
|
- HDFS-4146. Use getter and setter in INodeFileWithLink to access blocks and
|
|
|
- initialize root directory as snapshottable. (szetszwo)
|
|
|
+ HDFS-4569. Small image transfer related cleanups.
|
|
|
+ (Andrew Wang via suresh)
|
|
|
|
|
|
- HDFS-4149. Implement the disallowSnapshot(..) in FSNamesystem and add
|
|
|
- resetSnapshottable(..) to SnapshotManager. (szetszwo)
|
|
|
+ HDFS-4521. Invalid network toploogies should not be cached. (Colin Patrick
|
|
|
+ McCabe via atm)
|
|
|
|
|
|
- HDFS-4147. When there is a snapshot in a subtree, deletion of the subtree
|
|
|
- should fail. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4246. The exclude node list should be more forgiving, for each output
|
|
|
+ stream. (harsh via atm)
|
|
|
|
|
|
- HDFS-4150. Update the inode in the block map when a snapshotted file or a
|
|
|
- snapshot file is deleted. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4635. Move BlockManager#computeCapacity to LightWeightGSet. (suresh)
|
|
|
|
|
|
- HDFS-4159. Rename should fail when the destination directory is
|
|
|
- snapshottable and has snapshots. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4621. Additional logging to help diagnose slow QJM syncs. (todd)
|
|
|
|
|
|
- HDFS-4170. Add snapshot information to INodesInPath. (szetszwo)
|
|
|
+ HDFS-4618. Default transaction interval for checkpoints is too low. (todd)
|
|
|
|
|
|
- HDFS-4177. Add a snapshot parameter to INodeDirectory.getChildrenList() for
|
|
|
- selecting particular snapshot children list views. (szetszwo)
|
|
|
+ HDFS-4525. Provide an API for knowing that whether file is closed or not.
|
|
|
+ (SreeHari via umamahesh)
|
|
|
|
|
|
- HDFS-4148. Disallow write/modify operations on files and directories in a
|
|
|
- snapshot. (Brandon Li via suresh)
|
|
|
+ HDFS-3940. Add Gset#clear method and clear the block map when namenode is
|
|
|
+ shutdown. (suresh)
|
|
|
|
|
|
- HDFS-4188. Add Snapshot.ID_COMPARATOR for comparing IDs and fix a bug in
|
|
|
- ReadOnlyList.Util.binarySearch(..). (szetszwo)
|
|
|
+ HDFS-4679. Namenode operation checks should be done in a consistent
|
|
|
+ manner. (suresh)
|
|
|
|
|
|
- HDFS-4187. Add tests for replication handling in snapshots. (Jing Zhao via
|
|
|
- szetszwo)
|
|
|
+ HDFS-4693. Some test cases in TestCheckpoint do not clean up after
|
|
|
+ themselves. (Arpit Agarwal, suresh via suresh)
|
|
|
|
|
|
- HDFS-4196. Support renaming of snapshots. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-3817. Avoid printing SafeModeException stack trace.
|
|
|
+ (Brandon Li via suresh)
|
|
|
|
|
|
- HDFS-4175. Additional snapshot tests for more complicated directory
|
|
|
- structure and modifications. (Jing Zhao via suresh)
|
|
|
+ HDFS-4124. Refactor INodeDirectory#getExistingPathINodes() to enable
|
|
|
+ returning more than INode array. (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-4293. Fix TestSnapshot failure. (Jing Zhao via suresh)
|
|
|
+ HDFS-4151. Change the methods in FSDirectory to pass INodesInPath instead
|
|
|
+ of INode[] as a parameter. (szetszwo)
|
|
|
|
|
|
- HDFS-4317. Change INode and its subclasses to support HDFS-4103. (szetszwo)
|
|
|
+ HDFS-4129. Add utility methods to dump NameNode in memory tree for
|
|
|
+ testing. (szetszwo via suresh)
|
|
|
|
|
|
- HDFS-4103. Support O(1) snapshot creation. (szetszwo)
|
|
|
+ HDFS-4152. Add a new class BlocksMapUpdateInfo for the parameter in
|
|
|
+ INode.collectSubtreeBlocksAndClear(..). (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4330. Support snapshots up to the snapshot limit. (szetszwo)
|
|
|
+ HDFS-4206. Change the fields in INode and its subclasses to private.
|
|
|
+ (szetszwo)
|
|
|
|
|
|
- HDFS-4357. Fix a bug that if an inode is replaced, further INode operations
|
|
|
- should apply to the new inode. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4215. Remove locking from addToParent(..) since it is used in image
|
|
|
+ loading, and add INode.isFile(). (szetszwo)
|
|
|
|
|
|
- HDFS-4230. Support listing of all the snapshottable directories. (Jing Zhao
|
|
|
+ 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-4244. Support snapshot deletion. (Jing Zhao via szetszwo)
|
|
|
-
|
|
|
- HDFS-4245. Include snapshot related operations in TestOfflineEditsViewer.
|
|
|
- (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4209. Clean up the addNode/addChild/addChildNoQuotaCheck methods in
|
|
|
+ FSDirectory and INodeDirectory. (szetszwo)
|
|
|
|
|
|
- HDFS-4395. In INodeDirectorySnapshottable's constructor, the passed-in dir
|
|
|
- could be an INodeDirectoryWithSnapshot. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4346. Add SequentialNumber as a base class for INodeId and
|
|
|
+ GenerationStamp. (szetszwo)
|
|
|
|
|
|
- HDFS-4397. Fix a bug in INodeDirectoryWithSnapshot.Diff.combinePostDiff(..)
|
|
|
- that it may put the wrong node into the deleted list. (szetszwo)
|
|
|
+ HDFS-4721. Speed up lease recovery by avoiding stale datanodes and choosing
|
|
|
+ the datanode with the most recent heartbeat as the primary. (Varun Sharma
|
|
|
+ via szetszwo)
|
|
|
|
|
|
- HDFS-4407. Change INodeDirectoryWithSnapshot.Diff.combinePostDiff(..) to
|
|
|
- merge-sort like and keep the postDiff parameter unmodified. (szetszwo)
|
|
|
+ HDFS-4804. WARN when users set the block balanced preference percent below
|
|
|
+ 0.5 or above 1.0. (Stephen Chu via atm)
|
|
|
|
|
|
- HDFS-4098. Add FileWithSnapshot, INodeFileUnderConstructionWithSnapshot and
|
|
|
- INodeFileUnderConstructionSnapshot for supporting append to snapshotted
|
|
|
- files. (szetszwo)
|
|
|
+ HDFS-4698. Provide client-side metrics for remote reads, local reads, and
|
|
|
+ short-circuit reads. (Colin Patrick McCabe via atm)
|
|
|
|
|
|
- HDFS-4126. Add reading/writing snapshot information to FSImage.
|
|
|
- (Jing Zhao via suresh)
|
|
|
+ OPTIMIZATIONS
|
|
|
|
|
|
- HDFS-4436. Change INode.recordModification(..) to return only the current
|
|
|
- inode and remove the updateCircularList parameter from some methods in
|
|
|
- INodeDirectoryWithSnapshot.Diff. (szetszwo)
|
|
|
+ BUG FIXES
|
|
|
+
|
|
|
+ HDFS-4470. Several HDFS tests attempt file operations on invalid HDFS
|
|
|
+ paths when running on Windows. (Chris Nauroth via suresh)
|
|
|
|
|
|
- HDFS-4429. When the latest snapshot exists, INodeFileUnderConstruction
|
|
|
- should be replaced with INodeFileWithSnapshot but not INodeFile.
|
|
|
- (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4471. Namenode WebUI file browsing does not work with wildcard
|
|
|
+ addresses configured. (Andrew Wang via atm)
|
|
|
|
|
|
- HDFS-4441. Move INodeDirectoryWithSnapshot.Diff and the related classes to a
|
|
|
- package. (szetszwo)
|
|
|
+ HDFS-4342. Directories configured in dfs.namenode.edits.dir.required
|
|
|
+ but not in dfs.namenode.edits.dir are silently ignored. (Arpit Agarwal
|
|
|
+ via szetszwo)
|
|
|
|
|
|
- HDFS-4432. Support INodeFileUnderConstructionWithSnapshot in FSImage
|
|
|
- saving/loading. (Jing Zhao via suresh)
|
|
|
+ HDFS-4482. ReplicationMonitor thread can exit with NPE due to the race
|
|
|
+ between delete and replication of same file. (umamahesh)
|
|
|
|
|
|
- HDFS-4131. Add capability to namenode to get snapshot diff. (Jing Zhao via
|
|
|
- suresh)
|
|
|
+ HDFS-4269. Datanode rejects all datanode registrations from localhost
|
|
|
+ in single-node developer setup on Windows. (Chris Nauroth via suresh)
|
|
|
|
|
|
- HDFS-4447. Refactor INodeDirectoryWithSnapshot for supporting general INode
|
|
|
- diff lists. (szetszwo)
|
|
|
+ HDFS-4235. When outputting XML, OfflineEditsViewer can't handle some edits
|
|
|
+ containing non-ASCII strings. (Colin Patrick McCabe via atm)
|
|
|
|
|
|
- HDFS-4189. Renames the getMutableXxx methods to getXxx4Write and fix a bug
|
|
|
- that some getExistingPathINodes calls should be getINodesInPath4Write.
|
|
|
- (szetszwo)
|
|
|
+ HDFS-4541. Set hadoop.log.dir and hadoop.id.str when starting secure
|
|
|
+ datanode to write the logs to right dir by default. (Arpit Gupta via
|
|
|
+ suresh)
|
|
|
|
|
|
- HDFS-4361. When listing snapshottable directories, only return those
|
|
|
- where the user has permission to take snapshots. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4540. Namenode http server should use the web authentication
|
|
|
+ keytab for spnego principal. (Arpit Gupta via suresh)
|
|
|
|
|
|
- HDFS-4464. Combine collectSubtreeBlocksAndClear with deleteDiffsForSnapshot
|
|
|
- and rename it to destroySubtreeAndCollectBlocks. (szetszwo)
|
|
|
+ HDFS-4544. Error in deleting blocks should not do check disk, for
|
|
|
+ all types of errors. (Arpit Agarwal via suresh)
|
|
|
|
|
|
- HDFS-4414. Add support for getting snapshot diff from DistributedFileSystem.
|
|
|
- (Jing Zhao via suresh)
|
|
|
+ HDFS-4565. Use DFSUtil.getSpnegoKeytabKey() to get the spnego keytab key
|
|
|
+ in secondary namenode and namenode http server. (Arpit Gupta via suresh)
|
|
|
|
|
|
- HDFS-4446. Support file snapshots with diff lists. (szetszwo)
|
|
|
+ HDFS-4571. WebHDFS should not set the service hostname on the server side.
|
|
|
+ (tucu)
|
|
|
|
|
|
- HDFS-4480. Eliminate the file snapshot circular linked list. (szetszwo)
|
|
|
+ HDFS-4013. TestHftpURLTimeouts throws NPE. (Chao Shi via suresh)
|
|
|
|
|
|
- HDFS-4481. Change fsimage to support snapshot file diffs. (szetszwo)
|
|
|
+ HDFS-4592. Default values for access time precision are out of sync between
|
|
|
+ hdfs-default.xml and the code. (atm)
|
|
|
|
|
|
- HDFS-4500. Refactor snapshot INode methods. (szetszwo)
|
|
|
+ HDFS-4522. LightWeightGSet expects incrementing a volatile to be atomic.
|
|
|
+ (Colin Patrick McCabe via atm)
|
|
|
|
|
|
- HDFS-4487. Fix snapshot diff report for HDFS-4446. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4484. libwebhdfs compilation broken with gcc 4.6.2. (Colin Patrick
|
|
|
+ McCabe via atm)
|
|
|
|
|
|
- HDFS-4431. Support snapshot in OfflineImageViewer. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4595. When short circuit read is fails, DFSClient does not fallback
|
|
|
+ to regular reads. (suresh)
|
|
|
|
|
|
- HDFS-4503. Update computeContentSummary(..), spaceConsumedInTree(..) and
|
|
|
- diskspaceConsumed(..) in INode for snapshot. (szetszwo)
|
|
|
+ HDFS-4583. TestNodeCount fails. (Ivan Mitic via suresh)
|
|
|
|
|
|
- HDFS-4499. Fix file/directory/snapshot deletion for file diff. (Jing Zhao
|
|
|
- via szetszwo)
|
|
|
+ HDFS-4591. HA clients can fail to fail over while Standby NN is performing
|
|
|
+ long checkpoint. (atm)
|
|
|
|
|
|
- HDFS-4524. Update SnapshotManager#snapshottables when loading fsimage.
|
|
|
- (Jing Zhao via szetszwo)
|
|
|
+ HDFS-3277. fail over to loading a different FSImage if the first one we
|
|
|
+ try to load is corrupt. (Colin Patrick McCabe and Andrew Wang via atm)
|
|
|
|
|
|
- HDFS-4520. Support listing snapshots under a snapshottable directory using
|
|
|
- ls. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4596. Shutting down namenode during checkpointing can lead to md5sum
|
|
|
+ error. (Andrew Wang via atm)
|
|
|
|
|
|
- HDFS-4514. Add CLI for supporting snapshot rename, diff report, and
|
|
|
- snapshottable directory listing. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4614. FSNamesystem#getContentSummary should use getPermissionChecker
|
|
|
+ helper method. (atm)
|
|
|
|
|
|
- HDFS-4523. Fix INodeFile replacement, TestQuota and javac errors from trunk
|
|
|
- merge. (szetszwo)
|
|
|
+ HDFS-4620. Documentation for dfs.namenode.rpc-address specifies wrong
|
|
|
+ format. (Sandy Ryza via atm)
|
|
|
|
|
|
- HDFS-4507. Update quota verification for snapshots. (szetszwo)
|
|
|
+ HDFS-4609. TestAuditLogs should release log handles between tests.
|
|
|
+ (Ivan Mitic via szetszwo)
|
|
|
|
|
|
- HDFS-4545. With snapshots, FSDirectory.unprotectedSetReplication(..) always
|
|
|
- changes file replication but it may or may not changes block replication.
|
|
|
+ HDFS-4598. Fix the default value of ConcatSourcesParam and the WebHDFS doc.
|
|
|
(szetszwo)
|
|
|
|
|
|
- HDFS-4557. Fix FSDirectory#delete when INode#cleanSubtree returns 0.
|
|
|
- (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4655. DNA_FINALIZE is logged as being an unknown command by the DN
|
|
|
+ when received from the standby NN. (atm)
|
|
|
|
|
|
- HDFS-4579. Annotate snapshot tests. (Arpit Agarwal via suresh)
|
|
|
+ HDFS-4656. DN heartbeat loop can be briefly tight. (atm)
|
|
|
|
|
|
- HDFS-4574. Move Diff to the util package. (szetszwo)
|
|
|
+ HDFS-4658. Standby NN will log that it has received a block report "after
|
|
|
+ becoming active" (atm)
|
|
|
|
|
|
- HDFS-4563. Update namespace/diskspace usage after deleting snapshots.
|
|
|
- (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4646. createNNProxyWithClientProtocol ignores configured timeout
|
|
|
+ value (Jagane Sundar via cos)
|
|
|
|
|
|
- HDFS-4144. Create test for all snapshot-related metrics.
|
|
|
- (Jing Zhao via suresh)
|
|
|
+ HDFS-3981. Fix handling of FSN lock in getBlockLocations. (Xiaobo Peng
|
|
|
+ and todd via todd)
|
|
|
|
|
|
- HDFS-4556. Add snapshotdiff and LsSnapshottableDir tools to hdfs script.
|
|
|
- (Arpit Agarwal via szetszwo)
|
|
|
+ HDFS-4676. TestHDFSFileSystemContract should set MiniDFSCluster variable
|
|
|
+ to null to free up memory. (suresh)
|
|
|
|
|
|
- HDFS-4534. Add INodeReference in order to support rename with snapshots.
|
|
|
- (szetszwo)
|
|
|
+ HDFS-4669. TestBlockPoolManager fails using IBM java. (Tian Hong Wang via
|
|
|
+ suresh)
|
|
|
|
|
|
- HDFS-4616. Update the FilesDeleted metric while deleting file/dir in the
|
|
|
- current tree. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4643. Fix flakiness in TestQuorumJournalManager. (todd)
|
|
|
|
|
|
- HDFS-4627. Fix FSImageFormat#Loader NPE and synchronization issues.
|
|
|
- (Jing Zhao via suresh)
|
|
|
+ HDFS-4639. startFileInternal() should not increment generation stamp.
|
|
|
+ (Plamen Jeliazkov via shv)
|
|
|
|
|
|
- HDFS-4612. Not to use INode.getParent() when generating snapshot diff
|
|
|
- report. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4695. TestEditLog leaks open file handles between tests.
|
|
|
+ (Ivan Mitic via suresh)
|
|
|
|
|
|
- HDFS-4636. Update quota usage when deleting files/dirs that were created
|
|
|
- after taking the latest snapshot. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4737. JVM path embedded in fuse binaries. (Sean Mackrory via atm)
|
|
|
|
|
|
- HDFS-4648. For snapshot deletion, when merging the diff from to-delete
|
|
|
- snapshot to the prior snapshot, make sure files/directories created after
|
|
|
- the prior snapshot get deleted. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4739. NN can miscalculate the number of extra edit log segments to
|
|
|
+ retain. (atm)
|
|
|
|
|
|
- HDFS-4637. INodeDirectory#replaceSelf4Quota may incorrectly convert a newly
|
|
|
- created directory to an INodeDirectoryWithSnapshot. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4745. TestDataTransferKeepalive#testSlowReader has race condition that
|
|
|
+ causes sporadic failure. (Chris Nauroth via suresh)
|
|
|
|
|
|
- HDFS-4611. Update FSImage for INodeReference. (szetszwo)
|
|
|
+ HDFS-4768. File handle leak in datanode when a block pool is removed.
|
|
|
+ (Chris Nauroth via suresh)
|
|
|
|
|
|
- HDFS-4647. Rename should call setLocalName after an inode is removed from
|
|
|
- snapshots. (Arpit Agarwal via szetszwo)
|
|
|
+ HDFS-4748. MiniJournalCluster#restartJournalNode leaks resources, which
|
|
|
+ causes sporadic test failures. (Chris Nauroth via suresh)
|
|
|
|
|
|
- HDFS-4684. Use INode id for image serialization when writing INodeReference.
|
|
|
- (szetszwo)
|
|
|
+ HDFS-4733. Make HttpFS username pattern configurable. (tucu via atm)
|
|
|
|
|
|
- HDFS-4675. Fix rename across snapshottable directories. (Jing Zhao via
|
|
|
- szetszwo)
|
|
|
+ HDFS-4778. Fixes some issues that the first patch on HDFS-2576 missed.
|
|
|
+ (ddas)
|
|
|
|
|
|
- HDFS-4692. Use timestamp as default snapshot names. (szetszwo)
|
|
|
+ HDFS-4785. Concat operation does not remove concatenated files from
|
|
|
+ InodeMap. (suresh)
|
|
|
|
|
|
- HDFS-4666. Define ".snapshot" as a reserved inode name so that users cannot
|
|
|
- create a file/directory with ".snapshot" as the name. If ".snapshot" is
|
|
|
- used in a previous version of HDFS, it must be renamed before upgrade;
|
|
|
- otherwise, upgrade will fail. (szetszwo)
|
|
|
+ HDFS-4784. NPE in FSDirectory.resolvePath(). (Brandon Li via suresh)
|
|
|
|
|
|
- HDFS-4700. Fix the undo section of rename with snapshots. (Jing Zhao via
|
|
|
- szetszwo)
|
|
|
+ HDFS-4810. several HDFS HA tests have timeouts that are too short. (Chris
|
|
|
+ Nauroth via atm)
|
|
|
|
|
|
- HDFS-4529. Disallow concat when one of the src files is in some snapshot.
|
|
|
- (szetszwo)
|
|
|
+ HDFS-4799. Corrupt replica can be prematurely removed from
|
|
|
+ corruptReplicas map. (todd via kihwal)
|
|
|
|
|
|
- HDFS-4550. Refactor INodeDirectory.INodesInPath to a standalone class.
|
|
|
- (szetszwo)
|
|
|
+ HDFS-4751. TestLeaseRenewer#testThreadName flakes. (Andrew Wang via atm)
|
|
|
|
|
|
- HDFS-4707. Add snapshot methods to FilterFileSystem and fix findbugs
|
|
|
- warnings. (szetszwo)
|
|
|
+ HDFS-4533. start-dfs.sh ignores additional parameters besides -upgrade.
|
|
|
+ (Fengdong Yu via suresh)
|
|
|
|
|
|
- HDFS-4706. Do not replace root inode for disallowSnapshot. (szetszwo)
|
|
|
+ HDFS-4765. Permission check of symlink deletion incorrectly throws
|
|
|
+ UnresolvedLinkException. (Andrew Wang via atm)
|
|
|
|
|
|
- HDFS-4717. Change the path parameter type of the snapshot methods in
|
|
|
- HdfsAdmin from String to Path. (szetszwo)
|
|
|
+ HDFS-4300. TransferFsImage.downloadEditsToStorage should use a tmp file for
|
|
|
+ destination. (Andrew Wang via atm)
|
|
|
|
|
|
- HDFS-4708. Add snapshot user documentation. (szetszwo)
|
|
|
+ HDFS-4813. Add volatile to BlocksMap.blocks so that the replication thread
|
|
|
+ can see the updated value. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4726. Fix test failures after merging the INodeId-INode mapping
|
|
|
- from trunk. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-3180. Add socket timeouts to WebHdfsFileSystem. (Chris Nauroth via
|
|
|
+ szetszwo)
|
|
|
|
|
|
- HDFS-4727. Update inodeMap after deleting files/directories/snapshots.
|
|
|
- (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4787. Create a new HdfsConfiguration before each TestDFSClientRetries
|
|
|
+ testcases. (Tian Hong Wang via atm)
|
|
|
|
|
|
- HDFS-4719. Remove AbstractINodeDiff.Factory and move its methods to
|
|
|
- AbstractINodeDiffList. (Arpit Agarwal via szetszwo)
|
|
|
+ HDFS-4830. Typo in config settings for AvailableSpaceVolumeChoosingPolicy
|
|
|
+ in hdfs-default.xml. (atm)
|
|
|
|
|
|
- HDFS-4735. DisallowSnapshot throws IllegalStateException for nested
|
|
|
- snapshottable directories. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4824. FileInputStreamCache.close leaves dangling reference to
|
|
|
+ FileInputStreamCache.cacheCleaner. (Colin Patrick McCabe via todd)
|
|
|
|
|
|
- HDFS-4738. Changes AbstractINodeDiff to implement Comparable<Integer>, and
|
|
|
- fix javadoc and other warnings. (szetszwo)
|
|
|
+ HDFS-4298. StorageRetentionManager spews warnings when used with QJM. (atm)
|
|
|
|
|
|
- HDFS-4686. Update quota computation for rename and INodeReference.
|
|
|
- (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4725. Fix HDFS file handle leaks in FSEditLog, NameNode,
|
|
|
+ OfflineEditsBinaryLoader and some tests. (Chris Nauroth via szetszwo)
|
|
|
|
|
|
- HDFS-4729. Fix OfflineImageViewer and permission checking for snapshot
|
|
|
- operations. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4825. webhdfs / httpfs tests broken because of min block size change.
|
|
|
+ (Andrew Wang via suresh)
|
|
|
|
|
|
- HDFS-4749. Use INodeId to identify the corresponding directory node in
|
|
|
- FSImage saving/loading. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4780. Use the correct relogin method for services. (Robert Parker via
|
|
|
+ kihwal)
|
|
|
|
|
|
- HDFS-4742. Fix appending to a renamed file with snapshot. (Jing Zhao via
|
|
|
- szetszwo)
|
|
|
+ HDFS-4827. Slight update to the implementation of API for handling favored
|
|
|
+ nodes in DFSClient (ddas)
|
|
|
|
|
|
- HDFS-4755. Fix AccessControlException message and moves "implements
|
|
|
- LinkedElement" from INode to INodeWithAdditionalFields. (szetszwo)
|
|
|
+ HDFS-4865. Remove sub resource warning from httpfs log at startup time.
|
|
|
+ (ywskycn via tucu)
|
|
|
|
|
|
- HDFS-4650. Fix a bug in FSDirectory and add more unit tests for rename with
|
|
|
- existence of snapshottable directories and snapshots. (Jing Zhao via
|
|
|
- szetszwo)
|
|
|
+ BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
|
|
|
|
|
|
- HDFS-4650. When passing two non-existing snapshot names to snapshotDiff, it
|
|
|
- returns success if the names are the same. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
|
|
|
+ (Colin Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-4354. Create DomainSocket and DomainPeer and associated unit tests.
|
|
|
+ (Colin Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-4356. BlockReaderLocal should use passed file descriptors rather than paths.
|
|
|
+ (Colin Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-4388. DomainSocket should throw AsynchronousCloseException when appropriate.
|
|
|
+ (Colin Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-4390. Bypass UNIX domain socket unit tests when they cannot be run.
|
|
|
+ (Colin Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-4400. DFSInputStream#getBlockReader: last retries should ignore the cache
|
|
|
+ (Colin Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-4401. Fix bug in DomainSocket path validation
|
|
|
+ (Colin Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-4402. Some small DomainSocket fixes: avoid findbugs warning, change
|
|
|
+ log level, etc. (Colin Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-4418. increase default FileInputStreamCache size (todd)
|
|
|
+
|
|
|
+ HDFS-4416. Rename dfs.datanode.domain.socket.path to dfs.domain.socket.path
|
|
|
+ (Colin Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-4417. Fix case where local reads get disabled incorrectly
|
|
|
+ (Colin Patrick McCabe and todd via todd)
|
|
|
+
|
|
|
+ HDFS-4433. Make TestPeerCache not flaky (Colin Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-4438. TestDomainSocket fails when system umask is set to 0002. (Colin
|
|
|
+ Patrick McCabe via atm)
|
|
|
+
|
|
|
+ HDFS-4440. Avoid annoying log message when dfs.domain.socket.path is not
|
|
|
+ set. (Colin Patrick McCabe via atm)
|
|
|
+
|
|
|
+ HDFS-4473. Don't create domain socket unless we need it. (Colin Patrick McCabe via atm)
|
|
|
+
|
|
|
+ HDFS-4485. DN should chmod socket path a+w. (Colin Patrick McCabe via atm)
|
|
|
+
|
|
|
+ HDFS-4453. Make a simple doc to describe the usage and design of the
|
|
|
+ shortcircuit read feature. (Colin Patrick McCabe via atm)
|
|
|
+
|
|
|
+ HDFS-4496. DFSClient: don't create a domain socket unless we need it (Colin
|
|
|
+ Patrick McCabe via todd)
|
|
|
+
|
|
|
+ HDFS-347: style cleanups (Colin Patrick McCabe via atm)
|
|
|
+
|
|
|
+ HDFS-4538. Allow use of legacy blockreader (Colin Patrick McCabe via todd)
|
|
|
|
|
|
- HDFS-4767. If a directory is snapshottable, do not replace it when clearing
|
|
|
- quota. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4661. A few little code cleanups of some HDFS-347-related code. (Colin
|
|
|
+ Patrick McCabe via atm)
|
|
|
|
|
|
- HDFS-4578. Restrict snapshot IDs to 24-bit wide. (Arpit Agarwal via
|
|
|
- szetszwo)
|
|
|
+ BREAKDOWN OF HADOOP-8562 and HDFS-3602 SUBTASKS AND RELATED JIRAS
|
|
|
|
|
|
- HDFS-4773. Fix bugs in quota usage computation and OfflineImageViewer.
|
|
|
- (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4145. Merge hdfs cmd line scripts from branch-1-win. (David Lao,
|
|
|
+ Bikas Saha, Lauren Yang, Chuan Liu, Thejas M Nair and Ivan Mitic via suresh)
|
|
|
|
|
|
- HDFS-4760. Update inodeMap after node replacement. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4163. HDFS distribution build fails on Windows. (Chris Nauroth via
|
|
|
+ suresh)
|
|
|
|
|
|
- HDFS-4758. Disallow nested snapshottable directories and unwrap
|
|
|
- RemoteException. (szetszwo)
|
|
|
+ HDFS-4316. branch-trunk-win contains test code accidentally added during
|
|
|
+ work on fixing tests on Windows. (Chris Nauroth via suresh)
|
|
|
|
|
|
- HDFS-4781. Fix a NullPointerException when listing .snapshot under
|
|
|
- a non-existing directory. (szetszwo)
|
|
|
+ HDFS-4297. Fix issues related to datanode concurrent reading and writing on
|
|
|
+ Windows. (Arpit Agarwal, Chuan Liu via suresh)
|
|
|
|
|
|
- HDFS-4791. Update and fix deletion of reference inode. (Jing Zhao via
|
|
|
- szetszwo)
|
|
|
+ HDFS-4573. Fix TestINodeFile on Windows. (Arpit Agarwal via suresh)
|
|
|
|
|
|
- HDFS-4798. Update computeContentSummary() for the reference nodes in
|
|
|
- snapshots. (szetszwo)
|
|
|
+ HDFS-4572. Fix TestJournal failures on Windows. (Arpit Agarwal via suresh)
|
|
|
|
|
|
- HDFS-4800. Fix INodeDirectoryWithSnapshot#cleanDeletedINode. (Jing Zhao via
|
|
|
- szetszwo)
|
|
|
+ HDFS-4287. HTTPFS tests fail on Windows. (Chris Nauroth via suresh)
|
|
|
|
|
|
- HDFS-4801. lsSnapshottableDir throws IllegalArgumentException when root is
|
|
|
- snapshottable. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4593. TestSaveNamespace fails on Windows. (Arpit Agarwal via suresh)
|
|
|
|
|
|
- HDFS-4802. Disallowing snapshot on / twice should throw SnapshotException
|
|
|
- but not IllegalStateException. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4582. TestHostsFiles fails on Windows. (Ivan Mitic via suresh)
|
|
|
|
|
|
- HDFS-4806. In INodeDirectoryWithSnapshot, use isInLatestSnapshot() to
|
|
|
- determine if an added/removed child should be recorded in the snapshot diff.
|
|
|
- (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4603. TestMiniDFSCluster fails on Windows. (Ivan Mitic via suresh)
|
|
|
|
|
|
- HDFS-4809. When a QuotaExceededException is thrown during rename, the quota
|
|
|
- usage should be subtracted back. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4604. TestJournalNode fails on Windows. (Ivan Mitic via suresh)
|
|
|
|
|
|
- HDFS-4842. Identify the correct prior snapshot when deleting a
|
|
|
- snapshot under a renamed subtree. (jing9)
|
|
|
+ HDFS-4607. In TestGetConf.testGetSpecificKey(), use a platform-specific
|
|
|
+ line separator; otherwise, it fails on Windows. (Ivan Mitic via szetszwo)
|
|
|
|
|
|
- HDFS-4846. Clean up snapshot CLI commands output stacktrace for invalid
|
|
|
- arguments. (Jing Zhao via brandonli)
|
|
|
+ HDFS-4625. Make TestNNWithQJM#testNewNamenodeTakesOverWriter work on
|
|
|
+ Windows. (Ivan Mitic via suresh)
|
|
|
|
|
|
- HDFS-4857. Snapshot.Root and AbstractINodeDiff#snapshotINode should not be
|
|
|
- put into INodeMap when loading FSImage. (jing9)
|
|
|
+ HDFS-4674. TestBPOfferService fails on Windows due to failure parsing
|
|
|
+ datanode data directory as URI. (Chris Nauroth via suresh)
|
|
|
|
|
|
- HDFS-4863. The root directory should be added to the snapshottable
|
|
|
- directory list while loading fsimage. (jing9)
|
|
|
+ HDFS-4615. Fix TestDFSShell failures on Windows. (Arpit Agarwal
|
|
|
+ via szetszwo)
|
|
|
|
|
|
- HDFS-4848. copyFromLocal and renaming a file to ".snapshot" should output
|
|
|
- that ".snapshot" is a reserved name. (Jing Zhao via brandonli)
|
|
|
+ HDFS-4584. Skip TestNNWithQJM.testNewNamenodeTakesOverWriter() on Windows.
|
|
|
+ (Arpit Agarwal via szetszwo)
|
|
|
|
|
|
-Release 2.0.5-beta - UNRELEASED
|
|
|
+ HDFS-4732. Fix TestDFSUpgradeFromImage which fails on Windows due to
|
|
|
+ failure to unpack old image tarball that contains hard links.
|
|
|
+ (Chris Nauroth via szetszwo)
|
|
|
|
|
|
- INCOMPATIBLE CHANGES
|
|
|
+ HDFS-4741. TestStorageRestore#testStorageRestoreFailure fails on Windows.
|
|
|
+ (Arpit Agarwal via suresh)
|
|
|
|
|
|
- HDFS-4305. Add a configurable limit on number of blocks per file, and min
|
|
|
- block size. (Andrew Wang via atm)
|
|
|
+ HDFS-4743. TestNNStorageRetentionManager fails on Windows.
|
|
|
+ (Chris Nauroth via suresh)
|
|
|
|
|
|
- HDFS-4434. Provide a mapping from INodeId to INode. (suresh)
|
|
|
+ HDFS-4740. Fixes for a few test failures on Windows.
|
|
|
+ (Arpit Agarwal via suresh)
|
|
|
|
|
|
- NEW FEATURES
|
|
|
+ HDFS-4722. TestGetConf#testFederation times out on Windows.
|
|
|
+ (Ivan Mitic via suresh)
|
|
|
|
|
|
- HDFS-1804. Add a new block-volume device choosing policy that looks at
|
|
|
- free space. (atm)
|
|
|
+ HDFS-4705. Address HDFS test failures on Windows because of invalid
|
|
|
+ dfs.namenode.name.dir. (Ivan Mitic via suresh)
|
|
|
|
|
|
- HDFS-4296. Reserve layout version for release 1.2.0. (suresh)
|
|
|
+ HDFS-4734. HDFS Tests that use ShellCommandFencer are broken on Windows.
|
|
|
+ (Arpit Agarwal via suresh)
|
|
|
|
|
|
- HDFS-4334. Add a unique id to INode. (Brandon Li via szetszwo)
|
|
|
+ HDFS-4610. Use common utils FileUtil#setReadable/Writable/Executable and
|
|
|
+ FileUtil#canRead/Write/Execute. (Ivan Mitic via suresh)
|
|
|
|
|
|
- HDFS-4339. Persist inode id in fsimage and editlog. (Brandon Li via
|
|
|
- suresh)
|
|
|
+ BREAKDOWN OF HDFS-2802 HDFS SNAPSHOT SUBTASKS AND RELATED JIRAS
|
|
|
|
|
|
- HDFS-4340. Update addBlock() to inculde inode id as additional argument.
|
|
|
- (Brandon Li via suresh)
|
|
|
+ HDFS-4076. Support snapshot of single files. (szetszwo)
|
|
|
|
|
|
- HDFS-4502. JsonUtil.toFileStatus(..) should check if the fileId property
|
|
|
- exists. (Brandon Li via suresh)
|
|
|
+ HDFS-4082. Add editlog opcodes for snapshot create and delete operations.
|
|
|
+ (suresh via szetszwo)
|
|
|
|
|
|
- HDFS-2576. Enhances the DistributedFileSystem's create API so that clients
|
|
|
- can specify favored datanodes for a file's blocks. (ddas)
|
|
|
+ HDFS-4086. Add editlog opcodes to allow and disallow snapshots on a
|
|
|
+ directory. (Brandon Li via suresh)
|
|
|
|
|
|
- HDFS-347. DFS read performance suboptimal when client co-located on nodes
|
|
|
- with data. (Colin Patrick McCabe via todd and atm)
|
|
|
+ HDFS-4083. Protocol changes for snapshots. (suresh)
|
|
|
|
|
|
- HADOOP-8562. Enhancements to support Hadoop on Windows Server and Windows
|
|
|
- Azure environments. (See breakdown of tasks below for subtasks and
|
|
|
- contributors)
|
|
|
-
|
|
|
- IMPROVEMENTS
|
|
|
+ HDFS-4077. Add support for Snapshottable Directory. (szetszwo via suresh)
|
|
|
|
|
|
- HDFS-4222. NN is unresponsive and loses heartbeats from DNs when
|
|
|
- configured to use LDAP and LDAP has issues. (Xiaobo Peng, suresh)
|
|
|
+ HDFS-4087. Protocol changes for listSnapshots functionality.
|
|
|
+ (Brandon Li via suresh)
|
|
|
|
|
|
- HDFS-4304. Make FSEditLogOp.MAX_OP_SIZE configurable. (Colin Patrick
|
|
|
- McCabe via atm)
|
|
|
+ HDFS-4079. Add SnapshotManager which maintains a list for all the
|
|
|
+ snapshottable directories and supports snapshot methods such as setting a
|
|
|
+ directory to snapshottable and creating a snapshot. (szetszwo)
|
|
|
|
|
|
- HDFS-4518. Finer grained metrics for HDFS capacity.
|
|
|
- (Arpit Agarwal via suresh)
|
|
|
+ HDFS-4078. Handle replication in snapshots. (szetszwo)
|
|
|
|
|
|
- HDFS-4519. Support overriding jsvc binary and log file locations
|
|
|
- when launching secure datanode. (Chris Nauroth via suresh)
|
|
|
+ HDFS-4084. Provide CLI support to allow and disallow snapshot
|
|
|
+ on a directory. (Brondon Li via suresh)
|
|
|
|
|
|
- HDFS-4569. Small image transfer related cleanups.
|
|
|
- (Andrew Wang via suresh)
|
|
|
+ HDFS-4091. Add snapshot quota to limit the number of snapshots allowed.
|
|
|
+ (szetszwo)
|
|
|
|
|
|
- HDFS-4521. Invalid network toploogies should not be cached. (Colin Patrick
|
|
|
- McCabe via atm)
|
|
|
+ HDFS-4097. Provide CLI support for createSnapshot. (Brandon Li via suresh)
|
|
|
|
|
|
- HDFS-4246. The exclude node list should be more forgiving, for each output
|
|
|
- stream. (harsh via atm)
|
|
|
+ HDFS-4092. Update file deletion logic for snapshot so that the current inode
|
|
|
+ is removed from the circular linked list; and if some blocks at the end of
|
|
|
+ the block list no longer belong to any other inode, collect them and update
|
|
|
+ the block list. (szetszwo)
|
|
|
|
|
|
- HDFS-4635. Move BlockManager#computeCapacity to LightWeightGSet. (suresh)
|
|
|
+ HDFS-4111. Support snapshot of subtrees. (szetszwo via suresh)
|
|
|
|
|
|
- HDFS-4621. Additional logging to help diagnose slow QJM syncs. (todd)
|
|
|
+ HDFS-4119. Complete the allowSnapshot code and add a test for it. (szetszwo)
|
|
|
|
|
|
- HDFS-4618. Default transaction interval for checkpoints is too low. (todd)
|
|
|
+ HDFS-4133. Add testcases for testing basic snapshot functionalities.
|
|
|
+ (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-4525. Provide an API for knowing that whether file is closed or not.
|
|
|
- (SreeHari via umamahesh)
|
|
|
+ HDFS-4116. Add auditlog for some snapshot operations. (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-3940. Add Gset#clear method and clear the block map when namenode is
|
|
|
- shutdown. (suresh)
|
|
|
+ HDFS-4095. Add some snapshot related metrics. (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-4679. Namenode operation checks should be done in a consistent
|
|
|
- manner. (suresh)
|
|
|
+ HDFS-4141. Support directory diff - the difference between the current state
|
|
|
+ and a previous snapshot of an INodeDirectory. (szetszwo)
|
|
|
|
|
|
- HDFS-4693. Some test cases in TestCheckpoint do not clean up after
|
|
|
- themselves. (Arpit Agarwal, suresh via suresh)
|
|
|
+ HDFS-4146. Use getter and setter in INodeFileWithLink to access blocks and
|
|
|
+ initialize root directory as snapshottable. (szetszwo)
|
|
|
|
|
|
- HDFS-3817. Avoid printing SafeModeException stack trace.
|
|
|
- (Brandon Li via suresh)
|
|
|
+ HDFS-4149. Implement the disallowSnapshot(..) in FSNamesystem and add
|
|
|
+ resetSnapshottable(..) to SnapshotManager. (szetszwo)
|
|
|
|
|
|
- HDFS-4124. Refactor INodeDirectory#getExistingPathINodes() to enable
|
|
|
- returning more than INode array. (Jing Zhao via suresh)
|
|
|
+ HDFS-4147. When there is a snapshot in a subtree, deletion of the subtree
|
|
|
+ should fail. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4151. Change the methods in FSDirectory to pass INodesInPath instead
|
|
|
- of INode[] as a parameter. (szetszwo)
|
|
|
+ HDFS-4150. Update the inode in the block map when a snapshotted file or a
|
|
|
+ snapshot file is deleted. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4129. Add utility methods to dump NameNode in memory tree for
|
|
|
- testing. (szetszwo via suresh)
|
|
|
+ HDFS-4159. Rename should fail when the destination directory is
|
|
|
+ snapshottable and has snapshots. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4152. Add a new class BlocksMapUpdateInfo for the parameter in
|
|
|
- INode.collectSubtreeBlocksAndClear(..). (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4170. Add snapshot information to INodesInPath. (szetszwo)
|
|
|
|
|
|
- HDFS-4206. Change the fields in INode and its subclasses to private.
|
|
|
- (szetszwo)
|
|
|
+ HDFS-4177. Add a snapshot parameter to INodeDirectory.getChildrenList() for
|
|
|
+ selecting particular snapshot children list views. (szetszwo)
|
|
|
|
|
|
- HDFS-4215. Remove locking from addToParent(..) since it is used in image
|
|
|
- loading, and add INode.isFile(). (szetszwo)
|
|
|
+ HDFS-4148. Disallow write/modify operations on files and directories in a
|
|
|
+ snapshot. (Brandon Li via suresh)
|
|
|
|
|
|
- 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-4188. Add Snapshot.ID_COMPARATOR for comparing IDs and fix a bug in
|
|
|
+ ReadOnlyList.Util.binarySearch(..). (szetszwo)
|
|
|
|
|
|
- HDFS-4209. Clean up the addNode/addChild/addChildNoQuotaCheck methods in
|
|
|
- FSDirectory and INodeDirectory. (szetszwo)
|
|
|
+ HDFS-4187. Add tests for replication handling in snapshots. (Jing Zhao via
|
|
|
+ szetszwo)
|
|
|
|
|
|
- HDFS-4346. Add SequentialNumber as a base class for INodeId and
|
|
|
- GenerationStamp. (szetszwo)
|
|
|
+ HDFS-4196. Support renaming of snapshots. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4721. Speed up lease recovery by avoiding stale datanodes and choosing
|
|
|
- the datanode with the most recent heartbeat as the primary. (Varun Sharma
|
|
|
- via szetszwo)
|
|
|
+ HDFS-4175. Additional snapshot tests for more complicated directory
|
|
|
+ structure and modifications. (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-4804. WARN when users set the block balanced preference percent below
|
|
|
- 0.5 or above 1.0. (Stephen Chu via atm)
|
|
|
+ HDFS-4293. Fix TestSnapshot failure. (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-4698. Provide client-side metrics for remote reads, local reads, and
|
|
|
- short-circuit reads. (Colin Patrick McCabe via atm)
|
|
|
+ HDFS-4317. Change INode and its subclasses to support HDFS-4103. (szetszwo)
|
|
|
|
|
|
- OPTIMIZATIONS
|
|
|
+ HDFS-4103. Support O(1) snapshot creation. (szetszwo)
|
|
|
|
|
|
- BUG FIXES
|
|
|
-
|
|
|
- HDFS-4470. Several HDFS tests attempt file operations on invalid HDFS
|
|
|
- paths when running on Windows. (Chris Nauroth via suresh)
|
|
|
+ HDFS-4330. Support snapshots up to the snapshot limit. (szetszwo)
|
|
|
|
|
|
- HDFS-4471. Namenode WebUI file browsing does not work with wildcard
|
|
|
- addresses configured. (Andrew Wang via atm)
|
|
|
+ HDFS-4357. Fix a bug that if an inode is replaced, further INode operations
|
|
|
+ should apply to the new inode. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4342. Directories configured in dfs.namenode.edits.dir.required
|
|
|
- but not in dfs.namenode.edits.dir are silently ignored. (Arpit Agarwal
|
|
|
+ HDFS-4230. Support listing of all the snapshottable directories. (Jing Zhao
|
|
|
via szetszwo)
|
|
|
|
|
|
- HDFS-4482. ReplicationMonitor thread can exit with NPE due to the race
|
|
|
- between delete and replication of same file. (umamahesh)
|
|
|
+ HDFS-4244. Support snapshot deletion. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4269. Datanode rejects all datanode registrations from localhost
|
|
|
- in single-node developer setup on Windows. (Chris Nauroth via suresh)
|
|
|
+ HDFS-4245. Include snapshot related operations in TestOfflineEditsViewer.
|
|
|
+ (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4235. When outputting XML, OfflineEditsViewer can't handle some edits
|
|
|
- containing non-ASCII strings. (Colin Patrick McCabe via atm)
|
|
|
+ HDFS-4395. In INodeDirectorySnapshottable's constructor, the passed-in dir
|
|
|
+ could be an INodeDirectoryWithSnapshot. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4541. Set hadoop.log.dir and hadoop.id.str when starting secure
|
|
|
- datanode to write the logs to right dir by default. (Arpit Gupta via
|
|
|
- suresh)
|
|
|
+ HDFS-4397. Fix a bug in INodeDirectoryWithSnapshot.Diff.combinePostDiff(..)
|
|
|
+ that it may put the wrong node into the deleted list. (szetszwo)
|
|
|
|
|
|
- HDFS-4540. Namenode http server should use the web authentication
|
|
|
- keytab for spnego principal. (Arpit Gupta via suresh)
|
|
|
+ HDFS-4407. Change INodeDirectoryWithSnapshot.Diff.combinePostDiff(..) to
|
|
|
+ merge-sort like and keep the postDiff parameter unmodified. (szetszwo)
|
|
|
|
|
|
- HDFS-4544. Error in deleting blocks should not do check disk, for
|
|
|
- all types of errors. (Arpit Agarwal via suresh)
|
|
|
+ HDFS-4098. Add FileWithSnapshot, INodeFileUnderConstructionWithSnapshot and
|
|
|
+ INodeFileUnderConstructionSnapshot for supporting append to snapshotted
|
|
|
+ files. (szetszwo)
|
|
|
|
|
|
- HDFS-4565. Use DFSUtil.getSpnegoKeytabKey() to get the spnego keytab key
|
|
|
- in secondary namenode and namenode http server. (Arpit Gupta via suresh)
|
|
|
+ HDFS-4126. Add reading/writing snapshot information to FSImage.
|
|
|
+ (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-4571. WebHDFS should not set the service hostname on the server side.
|
|
|
- (tucu)
|
|
|
+ HDFS-4436. Change INode.recordModification(..) to return only the current
|
|
|
+ inode and remove the updateCircularList parameter from some methods in
|
|
|
+ INodeDirectoryWithSnapshot.Diff. (szetszwo)
|
|
|
|
|
|
- HDFS-4013. TestHftpURLTimeouts throws NPE. (Chao Shi via suresh)
|
|
|
+ HDFS-4429. When the latest snapshot exists, INodeFileUnderConstruction
|
|
|
+ should be replaced with INodeFileWithSnapshot but not INodeFile.
|
|
|
+ (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4592. Default values for access time precision are out of sync between
|
|
|
- hdfs-default.xml and the code. (atm)
|
|
|
+ HDFS-4441. Move INodeDirectoryWithSnapshot.Diff and the related classes to a
|
|
|
+ package. (szetszwo)
|
|
|
|
|
|
- HDFS-4522. LightWeightGSet expects incrementing a volatile to be atomic.
|
|
|
- (Colin Patrick McCabe via atm)
|
|
|
+ HDFS-4432. Support INodeFileUnderConstructionWithSnapshot in FSImage
|
|
|
+ saving/loading. (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-4484. libwebhdfs compilation broken with gcc 4.6.2. (Colin Patrick
|
|
|
- McCabe via atm)
|
|
|
+ HDFS-4131. Add capability to namenode to get snapshot diff. (Jing Zhao via
|
|
|
+ suresh)
|
|
|
|
|
|
- HDFS-4595. When short circuit read is fails, DFSClient does not fallback
|
|
|
- to regular reads. (suresh)
|
|
|
+ HDFS-4447. Refactor INodeDirectoryWithSnapshot for supporting general INode
|
|
|
+ diff lists. (szetszwo)
|
|
|
|
|
|
- HDFS-4583. TestNodeCount fails. (Ivan Mitic via suresh)
|
|
|
+ HDFS-4189. Renames the getMutableXxx methods to getXxx4Write and fix a bug
|
|
|
+ that some getExistingPathINodes calls should be getINodesInPath4Write.
|
|
|
+ (szetszwo)
|
|
|
|
|
|
- HDFS-4591. HA clients can fail to fail over while Standby NN is performing
|
|
|
- long checkpoint. (atm)
|
|
|
+ HDFS-4361. When listing snapshottable directories, only return those
|
|
|
+ where the user has permission to take snapshots. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-3277. fail over to loading a different FSImage if the first one we
|
|
|
- try to load is corrupt. (Colin Patrick McCabe and Andrew Wang via atm)
|
|
|
+ HDFS-4464. Combine collectSubtreeBlocksAndClear with deleteDiffsForSnapshot
|
|
|
+ and rename it to destroySubtreeAndCollectBlocks. (szetszwo)
|
|
|
|
|
|
- HDFS-4596. Shutting down namenode during checkpointing can lead to md5sum
|
|
|
- error. (Andrew Wang via atm)
|
|
|
+ HDFS-4414. Add support for getting snapshot diff from DistributedFileSystem.
|
|
|
+ (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-4614. FSNamesystem#getContentSummary should use getPermissionChecker
|
|
|
- helper method. (atm)
|
|
|
+ HDFS-4446. Support file snapshots with diff lists. (szetszwo)
|
|
|
|
|
|
- HDFS-4620. Documentation for dfs.namenode.rpc-address specifies wrong
|
|
|
- format. (Sandy Ryza via atm)
|
|
|
+ HDFS-4480. Eliminate the file snapshot circular linked list. (szetszwo)
|
|
|
|
|
|
- HDFS-4609. TestAuditLogs should release log handles between tests.
|
|
|
- (Ivan Mitic via szetszwo)
|
|
|
+ HDFS-4481. Change fsimage to support snapshot file diffs. (szetszwo)
|
|
|
|
|
|
- HDFS-4598. Fix the default value of ConcatSourcesParam and the WebHDFS doc.
|
|
|
- (szetszwo)
|
|
|
+ HDFS-4500. Refactor snapshot INode methods. (szetszwo)
|
|
|
|
|
|
- HDFS-4655. DNA_FINALIZE is logged as being an unknown command by the DN
|
|
|
- when received from the standby NN. (atm)
|
|
|
+ HDFS-4487. Fix snapshot diff report for HDFS-4446. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4656. DN heartbeat loop can be briefly tight. (atm)
|
|
|
+ HDFS-4431. Support snapshot in OfflineImageViewer. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4658. Standby NN will log that it has received a block report "after
|
|
|
- becoming active" (atm)
|
|
|
+ HDFS-4503. Update computeContentSummary(..), spaceConsumedInTree(..) and
|
|
|
+ diskspaceConsumed(..) in INode for snapshot. (szetszwo)
|
|
|
|
|
|
- HDFS-4646. createNNProxyWithClientProtocol ignores configured timeout
|
|
|
- value (Jagane Sundar via cos)
|
|
|
+ HDFS-4499. Fix file/directory/snapshot deletion for file diff. (Jing Zhao
|
|
|
+ via szetszwo)
|
|
|
|
|
|
- HDFS-3981. Fix handling of FSN lock in getBlockLocations. (Xiaobo Peng
|
|
|
- and todd via todd)
|
|
|
+ HDFS-4524. Update SnapshotManager#snapshottables when loading fsimage.
|
|
|
+ (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4676. TestHDFSFileSystemContract should set MiniDFSCluster variable
|
|
|
- to null to free up memory. (suresh)
|
|
|
+ HDFS-4520. Support listing snapshots under a snapshottable directory using
|
|
|
+ ls. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4669. TestBlockPoolManager fails using IBM java. (Tian Hong Wang via
|
|
|
- suresh)
|
|
|
+ HDFS-4514. Add CLI for supporting snapshot rename, diff report, and
|
|
|
+ snapshottable directory listing. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4643. Fix flakiness in TestQuorumJournalManager. (todd)
|
|
|
+ HDFS-4523. Fix INodeFile replacement, TestQuota and javac errors from trunk
|
|
|
+ merge. (szetszwo)
|
|
|
|
|
|
- HDFS-4639. startFileInternal() should not increment generation stamp.
|
|
|
- (Plamen Jeliazkov via shv)
|
|
|
+ HDFS-4507. Update quota verification for snapshots. (szetszwo)
|
|
|
|
|
|
- HDFS-4695. TestEditLog leaks open file handles between tests.
|
|
|
- (Ivan Mitic via suresh)
|
|
|
+ HDFS-4545. With snapshots, FSDirectory.unprotectedSetReplication(..) always
|
|
|
+ changes file replication but it may or may not changes block replication.
|
|
|
+ (szetszwo)
|
|
|
|
|
|
- HDFS-4737. JVM path embedded in fuse binaries. (Sean Mackrory via atm)
|
|
|
+ HDFS-4557. Fix FSDirectory#delete when INode#cleanSubtree returns 0.
|
|
|
+ (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4739. NN can miscalculate the number of extra edit log segments to
|
|
|
- retain. (atm)
|
|
|
+ HDFS-4579. Annotate snapshot tests. (Arpit Agarwal via suresh)
|
|
|
|
|
|
- HDFS-4745. TestDataTransferKeepalive#testSlowReader has race condition that
|
|
|
- causes sporadic failure. (Chris Nauroth via suresh)
|
|
|
+ HDFS-4574. Move Diff to the util package. (szetszwo)
|
|
|
|
|
|
- HDFS-4768. File handle leak in datanode when a block pool is removed.
|
|
|
- (Chris Nauroth via suresh)
|
|
|
+ HDFS-4563. Update namespace/diskspace usage after deleting snapshots.
|
|
|
+ (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4748. MiniJournalCluster#restartJournalNode leaks resources, which
|
|
|
- causes sporadic test failures. (Chris Nauroth via suresh)
|
|
|
+ HDFS-4144. Create test for all snapshot-related metrics.
|
|
|
+ (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-4733. Make HttpFS username pattern configurable. (tucu via atm)
|
|
|
+ HDFS-4556. Add snapshotdiff and LsSnapshottableDir tools to hdfs script.
|
|
|
+ (Arpit Agarwal via szetszwo)
|
|
|
|
|
|
- HDFS-4778. Fixes some issues that the first patch on HDFS-2576 missed.
|
|
|
- (ddas)
|
|
|
+ HDFS-4534. Add INodeReference in order to support rename with snapshots.
|
|
|
+ (szetszwo)
|
|
|
|
|
|
- HDFS-4785. Concat operation does not remove concatenated files from
|
|
|
- InodeMap. (suresh)
|
|
|
+ HDFS-4616. Update the FilesDeleted metric while deleting file/dir in the
|
|
|
+ current tree. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4784. NPE in FSDirectory.resolvePath(). (Brandon Li via suresh)
|
|
|
+ HDFS-4627. Fix FSImageFormat#Loader NPE and synchronization issues.
|
|
|
+ (Jing Zhao via suresh)
|
|
|
|
|
|
- HDFS-4810. several HDFS HA tests have timeouts that are too short. (Chris
|
|
|
- Nauroth via atm)
|
|
|
+ HDFS-4612. Not to use INode.getParent() when generating snapshot diff
|
|
|
+ report. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4799. Corrupt replica can be prematurely removed from
|
|
|
- corruptReplicas map. (todd via kihwal)
|
|
|
+ HDFS-4636. Update quota usage when deleting files/dirs that were created
|
|
|
+ after taking the latest snapshot. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4751. TestLeaseRenewer#testThreadName flakes. (Andrew Wang via atm)
|
|
|
+ HDFS-4648. For snapshot deletion, when merging the diff from to-delete
|
|
|
+ snapshot to the prior snapshot, make sure files/directories created after
|
|
|
+ the prior snapshot get deleted. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4533. start-dfs.sh ignores additional parameters besides -upgrade.
|
|
|
- (Fengdong Yu via suresh)
|
|
|
+ HDFS-4637. INodeDirectory#replaceSelf4Quota may incorrectly convert a newly
|
|
|
+ created directory to an INodeDirectoryWithSnapshot. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4765. Permission check of symlink deletion incorrectly throws
|
|
|
- UnresolvedLinkException. (Andrew Wang via atm)
|
|
|
+ HDFS-4611. Update FSImage for INodeReference. (szetszwo)
|
|
|
|
|
|
- HDFS-4300. TransferFsImage.downloadEditsToStorage should use a tmp file for
|
|
|
- destination. (Andrew Wang via atm)
|
|
|
+ HDFS-4647. Rename should call setLocalName after an inode is removed from
|
|
|
+ snapshots. (Arpit Agarwal via szetszwo)
|
|
|
|
|
|
- HDFS-4813. Add volatile to BlocksMap.blocks so that the replication thread
|
|
|
- can see the updated value. (Jing Zhao via szetszwo)
|
|
|
+ HDFS-4684. Use INode id for image serialization when writing INodeReference.
|
|
|
+ (szetszwo)
|
|
|
|
|
|
- HDFS-3180. Add socket timeouts to WebHdfsFileSystem. (Chris Nauroth via
|
|
|
+ HDFS-4675. Fix rename across snapshottable directories. (Jing Zhao via
|
|
|
szetszwo)
|
|
|
|
|
|
- HDFS-4787. Create a new HdfsConfiguration before each TestDFSClientRetries
|
|
|
- testcases. (Tian Hong Wang via atm)
|
|
|
+ HDFS-4692. Use timestamp as default snapshot names. (szetszwo)
|
|
|
|
|
|
- HDFS-4830. Typo in config settings for AvailableSpaceVolumeChoosingPolicy
|
|
|
- in hdfs-default.xml. (atm)
|
|
|
+ HDFS-4666. Define ".snapshot" as a reserved inode name so that users cannot
|
|
|
+ create a file/directory with ".snapshot" as the name. If ".snapshot" is
|
|
|
+ used in a previous version of HDFS, it must be renamed before upgrade;
|
|
|
+ otherwise, upgrade will fail. (szetszwo)
|
|
|
|
|
|
- HDFS-4824. FileInputStreamCache.close leaves dangling reference to
|
|
|
- FileInputStreamCache.cacheCleaner. (Colin Patrick McCabe via todd)
|
|
|
+ HDFS-4700. Fix the undo section of rename with snapshots. (Jing Zhao via
|
|
|
+ szetszwo)
|
|
|
|
|
|
- HDFS-4298. StorageRetentionManager spews warnings when used with QJM. (atm)
|
|
|
+ HDFS-4529. Disallow concat when one of the src files is in some snapshot.
|
|
|
+ (szetszwo)
|
|
|
|
|
|
- HDFS-4725. Fix HDFS file handle leaks in FSEditLog, NameNode,
|
|
|
- OfflineEditsBinaryLoader and some tests. (Chris Nauroth via szetszwo)
|
|
|
+ HDFS-4550. Refactor INodeDirectory.INodesInPath to a standalone class.
|
|
|
+ (szetszwo)
|
|
|
|
|
|
- HDFS-4825. webhdfs / httpfs tests broken because of min block size change.
|
|
|
- (Andrew Wang via suresh)
|
|
|
+ HDFS-4707. Add snapshot methods to FilterFileSystem and fix findbugs
|
|
|
+ warnings. (szetszwo)
|
|
|
|
|
|
- HDFS-4780. Use the correct relogin method for services. (Robert Parker via
|
|
|
- kihwal)
|
|
|
+ HDFS-4706. Do not replace root inode for disallowSnapshot. (szetszwo)
|
|
|
|
|
|
- HDFS-4827. Slight update to the implementation of API for handling favored
|
|
|
- nodes in DFSClient (ddas)
|
|
|
+ HDFS-4717. Change the path parameter type of the snapshot methods in
|
|
|
+ HdfsAdmin from String to Path. (szetszwo)
|
|
|
|
|
|
- HDFS-4865. Remove sub resource warning from httpfs log at startup time.
|
|
|
- (ywskycn via tucu)
|
|
|
+ HDFS-4708. Add snapshot user documentation. (szetszwo)
|
|
|
|
|
|
- BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
|
|
|
+ HDFS-4726. Fix test failures after merging the INodeId-INode mapping
|
|
|
+ from trunk. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
|
|
|
- (Colin Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-4354. Create DomainSocket and DomainPeer and associated unit tests.
|
|
|
- (Colin Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-4356. BlockReaderLocal should use passed file descriptors rather than paths.
|
|
|
- (Colin Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-4388. DomainSocket should throw AsynchronousCloseException when appropriate.
|
|
|
- (Colin Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-4390. Bypass UNIX domain socket unit tests when they cannot be run.
|
|
|
- (Colin Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-4400. DFSInputStream#getBlockReader: last retries should ignore the cache
|
|
|
- (Colin Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-4401. Fix bug in DomainSocket path validation
|
|
|
- (Colin Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-4402. Some small DomainSocket fixes: avoid findbugs warning, change
|
|
|
- log level, etc. (Colin Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-4418. increase default FileInputStreamCache size (todd)
|
|
|
-
|
|
|
- HDFS-4416. Rename dfs.datanode.domain.socket.path to dfs.domain.socket.path
|
|
|
- (Colin Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-4417. Fix case where local reads get disabled incorrectly
|
|
|
- (Colin Patrick McCabe and todd via todd)
|
|
|
-
|
|
|
- HDFS-4433. Make TestPeerCache not flaky (Colin Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-4438. TestDomainSocket fails when system umask is set to 0002. (Colin
|
|
|
- Patrick McCabe via atm)
|
|
|
-
|
|
|
- HDFS-4440. Avoid annoying log message when dfs.domain.socket.path is not
|
|
|
- set. (Colin Patrick McCabe via atm)
|
|
|
-
|
|
|
- HDFS-4473. Don't create domain socket unless we need it. (Colin Patrick McCabe via atm)
|
|
|
-
|
|
|
- HDFS-4485. DN should chmod socket path a+w. (Colin Patrick McCabe via atm)
|
|
|
-
|
|
|
- HDFS-4453. Make a simple doc to describe the usage and design of the
|
|
|
- shortcircuit read feature. (Colin Patrick McCabe via atm)
|
|
|
-
|
|
|
- HDFS-4496. DFSClient: don't create a domain socket unless we need it (Colin
|
|
|
- Patrick McCabe via todd)
|
|
|
-
|
|
|
- HDFS-347: style cleanups (Colin Patrick McCabe via atm)
|
|
|
-
|
|
|
- HDFS-4538. Allow use of legacy blockreader (Colin Patrick McCabe via todd)
|
|
|
+ HDFS-4727. Update inodeMap after deleting files/directories/snapshots.
|
|
|
+ (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4661. A few little code cleanups of some HDFS-347-related code. (Colin
|
|
|
- Patrick McCabe via atm)
|
|
|
+ HDFS-4719. Remove AbstractINodeDiff.Factory and move its methods to
|
|
|
+ AbstractINodeDiffList. (Arpit Agarwal via szetszwo)
|
|
|
|
|
|
- BREAKDOWN OF HADOOP-8562 and HDFS-3602 SUBTASKS AND RELATED JIRAS
|
|
|
+ HDFS-4735. DisallowSnapshot throws IllegalStateException for nested
|
|
|
+ snapshottable directories. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4145. Merge hdfs cmd line scripts from branch-1-win. (David Lao,
|
|
|
- Bikas Saha, Lauren Yang, Chuan Liu, Thejas M Nair and Ivan Mitic via suresh)
|
|
|
+ HDFS-4738. Changes AbstractINodeDiff to implement Comparable<Integer>, and
|
|
|
+ fix javadoc and other warnings. (szetszwo)
|
|
|
|
|
|
- HDFS-4163. HDFS distribution build fails on Windows. (Chris Nauroth via
|
|
|
- suresh)
|
|
|
+ HDFS-4686. Update quota computation for rename and INodeReference.
|
|
|
+ (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4316. branch-trunk-win contains test code accidentally added during
|
|
|
- work on fixing tests on Windows. (Chris Nauroth via suresh)
|
|
|
+ HDFS-4729. Fix OfflineImageViewer and permission checking for snapshot
|
|
|
+ operations. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4297. Fix issues related to datanode concurrent reading and writing on
|
|
|
- Windows. (Arpit Agarwal, Chuan Liu via suresh)
|
|
|
+ HDFS-4749. Use INodeId to identify the corresponding directory node in
|
|
|
+ FSImage saving/loading. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4573. Fix TestINodeFile on Windows. (Arpit Agarwal via suresh)
|
|
|
+ HDFS-4742. Fix appending to a renamed file with snapshot. (Jing Zhao via
|
|
|
+ szetszwo)
|
|
|
|
|
|
- HDFS-4572. Fix TestJournal failures on Windows. (Arpit Agarwal via suresh)
|
|
|
+ HDFS-4755. Fix AccessControlException message and moves "implements
|
|
|
+ LinkedElement" from INode to INodeWithAdditionalFields. (szetszwo)
|
|
|
|
|
|
- HDFS-4287. HTTPFS tests fail on Windows. (Chris Nauroth via suresh)
|
|
|
+ HDFS-4650. Fix a bug in FSDirectory and add more unit tests for rename with
|
|
|
+ existence of snapshottable directories and snapshots. (Jing Zhao via
|
|
|
+ szetszwo)
|
|
|
|
|
|
- HDFS-4593. TestSaveNamespace fails on Windows. (Arpit Agarwal via suresh)
|
|
|
+ HDFS-4650. When passing two non-existing snapshot names to snapshotDiff, it
|
|
|
+ returns success if the names are the same. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4582. TestHostsFiles fails on Windows. (Ivan Mitic via suresh)
|
|
|
+ HDFS-4767. If a directory is snapshottable, do not replace it when clearing
|
|
|
+ quota. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4603. TestMiniDFSCluster fails on Windows. (Ivan Mitic via suresh)
|
|
|
+ HDFS-4578. Restrict snapshot IDs to 24-bit wide. (Arpit Agarwal via
|
|
|
+ szetszwo)
|
|
|
|
|
|
- HDFS-4604. TestJournalNode fails on Windows. (Ivan Mitic via suresh)
|
|
|
+ HDFS-4773. Fix bugs in quota usage computation and OfflineImageViewer.
|
|
|
+ (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4607. In TestGetConf.testGetSpecificKey(), use a platform-specific
|
|
|
- line separator; otherwise, it fails on Windows. (Ivan Mitic via szetszwo)
|
|
|
+ HDFS-4760. Update inodeMap after node replacement. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4625. Make TestNNWithQJM#testNewNamenodeTakesOverWriter work on
|
|
|
- Windows. (Ivan Mitic via suresh)
|
|
|
+ HDFS-4758. Disallow nested snapshottable directories and unwrap
|
|
|
+ RemoteException. (szetszwo)
|
|
|
|
|
|
- HDFS-4674. TestBPOfferService fails on Windows due to failure parsing
|
|
|
- datanode data directory as URI. (Chris Nauroth via suresh)
|
|
|
+ HDFS-4781. Fix a NullPointerException when listing .snapshot under
|
|
|
+ a non-existing directory. (szetszwo)
|
|
|
|
|
|
- HDFS-4615. Fix TestDFSShell failures on Windows. (Arpit Agarwal
|
|
|
- via szetszwo)
|
|
|
+ HDFS-4791. Update and fix deletion of reference inode. (Jing Zhao via
|
|
|
+ szetszwo)
|
|
|
|
|
|
- HDFS-4584. Skip TestNNWithQJM.testNewNamenodeTakesOverWriter() on Windows.
|
|
|
- (Arpit Agarwal via szetszwo)
|
|
|
+ HDFS-4798. Update computeContentSummary() for the reference nodes in
|
|
|
+ snapshots. (szetszwo)
|
|
|
|
|
|
- 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-4800. Fix INodeDirectoryWithSnapshot#cleanDeletedINode. (Jing Zhao via
|
|
|
+ szetszwo)
|
|
|
|
|
|
- HDFS-4741. TestStorageRestore#testStorageRestoreFailure fails on Windows.
|
|
|
- (Arpit Agarwal via suresh)
|
|
|
+ HDFS-4801. lsSnapshottableDir throws IllegalArgumentException when root is
|
|
|
+ snapshottable. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4743. TestNNStorageRetentionManager fails on Windows.
|
|
|
- (Chris Nauroth via suresh)
|
|
|
+ HDFS-4802. Disallowing snapshot on / twice should throw SnapshotException
|
|
|
+ but not IllegalStateException. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4740. Fixes for a few test failures on Windows.
|
|
|
- (Arpit Agarwal via suresh)
|
|
|
+ HDFS-4806. In INodeDirectoryWithSnapshot, use isInLatestSnapshot() to
|
|
|
+ determine if an added/removed child should be recorded in the snapshot diff.
|
|
|
+ (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4722. TestGetConf#testFederation times out on Windows.
|
|
|
- (Ivan Mitic via suresh)
|
|
|
+ HDFS-4809. When a QuotaExceededException is thrown during rename, the quota
|
|
|
+ usage should be subtracted back. (Jing Zhao via szetszwo)
|
|
|
|
|
|
- HDFS-4705. Address HDFS test failures on Windows because of invalid
|
|
|
- dfs.namenode.name.dir. (Ivan Mitic via suresh)
|
|
|
+ HDFS-4842. Identify the correct prior snapshot when deleting a
|
|
|
+ snapshot under a renamed subtree. (jing9)
|
|
|
|
|
|
- HDFS-4734. HDFS Tests that use ShellCommandFencer are broken on Windows.
|
|
|
- (Arpit Agarwal via suresh)
|
|
|
+ HDFS-4846. Clean up snapshot CLI commands output stacktrace for invalid
|
|
|
+ arguments. (Jing Zhao via brandonli)
|
|
|
|
|
|
- HDFS-4610. Use common utils FileUtil#setReadable/Writable/Executable and
|
|
|
- FileUtil#canRead/Write/Execute. (Ivan Mitic via suresh)
|
|
|
+ HDFS-4857. Snapshot.Root and AbstractINodeDiff#snapshotINode should not be
|
|
|
+ put into INodeMap when loading FSImage. (jing9)
|
|
|
+
|
|
|
+ HDFS-4863. The root directory should be added to the snapshottable
|
|
|
+ directory list while loading fsimage. (jing9)
|
|
|
+
|
|
|
+ HDFS-4848. copyFromLocal and renaming a file to ".snapshot" should output
|
|
|
+ that ".snapshot" is a reserved name. (Jing Zhao via brandonli)
|
|
|
|
|
|
Release 2.0.4-alpha - 2013-04-25
|
|
|
|