You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sh...@apache.org on 2018/09/18 02:06:24 UTC

[48/50] [abbrv] hadoop git commit: Merge commit 'eca1a4bfe952fc184fe90dde50bac9b0e5293568' into HDFS-12943

Merge commit 'eca1a4bfe952fc184fe90dde50bac9b0e5293568' into HDFS-12943

# Conflicts:
#	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
#	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9ade422b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9ade422b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9ade422b

Branch: refs/heads/HDFS-12943
Commit: 9ade422b3fb585fe4cbecdc241d6642dcd510e8e
Parents: e97b104 eca1a4b
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Mon Sep 17 17:49:26 2018 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Mon Sep 17 18:50:23 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  4 +--
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  2 +-
 .../java/org/apache/hadoop/hdfs/PeerCache.java  |  8 ++---
 .../hdfs/client/impl/BlockReaderFactory.java    | 12 +++----
 .../client/impl/BlockReaderLocalLegacy.java     |  2 +-
 .../hdfs/shortcircuit/ShortCircuitCache.java    |  4 +--
 .../hdfs/shortcircuit/ShortCircuitReplica.java  |  2 +-
 .../apache/hadoop/hdfs/util/IOUtilsClient.java  |  3 +-
 .../hadoop/hdfs/nfs/nfs3/OpenFileCtx.java       |  4 +--
 .../hadoop/hdfs/nfs/nfs3/OpenFileCtxCache.java  |  6 ++--
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  7 +++--
 .../org/apache/hadoop/hdfs/HdfsDtFetcher.java   |  7 +++--
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |  7 +++--
 .../apache/hadoop/hdfs/SWebHdfsDtFetcher.java   |  7 +++--
 .../apache/hadoop/hdfs/WebHdfsDtFetcher.java    |  7 +++--
 .../hadoop/hdfs/net/DomainPeerServer.java       |  6 ++--
 .../apache/hadoop/hdfs/net/TcpPeerServer.java   |  6 ++--
 .../hdfs/qjournal/client/AsyncLoggerSet.java    |  6 ++--
 .../qjournal/client/QuorumJournalManager.java   |  6 ++--
 .../qjournal/server/GetJournalEditServlet.java  |  7 +++--
 .../hadoop/hdfs/qjournal/server/Journal.java    | 12 +++----
 .../hdfs/qjournal/server/JournalNode.java       | 10 +++---
 .../qjournal/server/JournalNodeRpcServer.java   |  4 +--
 .../qjournal/server/JournaledEditsCache.java    |  2 +-
 .../token/block/BlockTokenSecretManager.java    |  7 +++--
 .../DelegationTokenSecretManager.java           |  8 ++---
 .../hadoop/hdfs/server/balancer/Balancer.java   |  8 ++---
 .../hadoop/hdfs/server/balancer/Dispatcher.java |  6 ++--
 .../hdfs/server/balancer/NameNodeConnector.java |  7 +++--
 .../AvailableSpaceBlockPlacementPolicy.java     |  8 ++---
 .../server/blockmanagement/DatanodeManager.java |  6 ++--
 .../server/blockmanagement/HostFileManager.java |  7 +++--
 .../hadoop/hdfs/server/common/JspHelper.java    |  6 ++--
 .../hdfs/server/common/MetricsLoggerTask.java   |  6 ++--
 .../apache/hadoop/hdfs/server/common/Util.java  |  7 +++--
 .../hdfs/server/datanode/DirectoryScanner.java  |  7 +++--
 .../server/datanode/ProfilingFileIoEvents.java  |  7 +++--
 .../server/datanode/ShortCircuitRegistry.java   |  7 +++--
 .../AvailableSpaceVolumeChoosingPolicy.java     |  7 +++--
 .../RoundRobinVolumeChoosingPolicy.java         |  7 +++--
 .../datanode/fsdataset/impl/BlockPoolSlice.java |  8 ++---
 .../impl/FsDatasetAsyncDiskService.java         |  7 +++--
 .../impl/RamDiskAsyncLazyPersistService.java    |  7 +++--
 .../fsdataset/impl/RamDiskReplicaTracker.java   |  7 +++--
 .../server/datanode/web/DatanodeHttpServer.java |  6 ++--
 .../web/RestCsrfPreventionFilterHandler.java    |  4 +--
 .../datanode/web/SimpleHttpProxyHandler.java    |  4 +--
 .../web/webhdfs/DataNodeUGIProvider.java        |  6 ++--
 .../datanode/web/webhdfs/ExceptionHandler.java  |  4 +--
 .../server/datanode/web/webhdfs/HdfsWriter.java |  8 ++---
 .../datanode/web/webhdfs/WebHdfsHandler.java    | 10 +++---
 .../apache/hadoop/hdfs/server/mover/Mover.java  | 12 +++----
 .../hadoop/hdfs/server/namenode/CachePool.java  |  2 --
 .../hdfs/server/namenode/CheckpointConf.java    |  7 +++--
 .../hdfs/server/namenode/Checkpointer.java      |  8 ++---
 .../ContentSummaryComputationContext.java       |  8 ++---
 .../hadoop/hdfs/server/namenode/DfsServlet.java |  7 +++--
 .../namenode/EditLogBackupOutputStream.java     |  7 +++--
 .../server/namenode/EditLogFileInputStream.java |  8 ++---
 .../namenode/EditLogFileOutputStream.java       | 11 ++++---
 .../hdfs/server/namenode/EditsDoubleBuffer.java |  7 +++--
 .../hdfs/server/namenode/FSEditLogAsync.java    |  8 ++---
 .../hdfs/server/namenode/FSEditLogLoader.java   |  7 +++--
 .../hadoop/hdfs/server/namenode/FSImage.java    |  9 +++---
 .../hdfs/server/namenode/FSImageFormat.java     |  6 ++--
 .../server/namenode/FSImageFormatPBINode.java   |  7 +++--
 ...FSImagePreTransactionalStorageInspector.java | 10 +++---
 .../FSImageTransactionalStorageInspector.java   |  6 ++--
 .../hdfs/server/namenode/FSNamesystem.java      |  2 +-
 .../server/namenode/FSPermissionChecker.java    |  6 ++--
 .../server/namenode/FileJournalManager.java     |  7 +++--
 .../hadoop/hdfs/server/namenode/INode.java      |  6 ++--
 .../hdfs/server/namenode/INodesInPath.java      |  6 ++--
 .../hdfs/server/namenode/ImageServlet.java      |  6 ++--
 .../hadoop/hdfs/server/namenode/JournalSet.java |  8 ++---
 .../server/namenode/MetaRecoveryContext.java    |  7 +++--
 .../namenode/NNStorageRetentionManager.java     |  6 ++--
 .../hdfs/server/namenode/NNUpgradeUtil.java     |  7 +++--
 .../hadoop/hdfs/server/namenode/NameCache.java  |  6 ++--
 .../namenode/NameNodeResourceChecker.java       |  7 +++--
 .../hdfs/server/namenode/NamenodeFsck.java      |  9 +++---
 .../namenode/RedundantEditLogInputStream.java   |  8 ++---
 .../hdfs/server/namenode/SecondaryNameNode.java | 16 +++++-----
 .../server/namenode/StartupProgressServlet.java |  2 +-
 .../hdfs/server/namenode/TransferFsImage.java   |  7 +++--
 .../server/namenode/ha/BootstrapStandby.java    | 13 ++++----
 .../hdfs/server/namenode/ha/EditLogTailer.java  |  8 ++---
 .../web/resources/NamenodeWebHdfsMethods.java   |  7 +++--
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |  6 ++--
 .../apache/hadoop/hdfs/tools/DFSHAAdmin.java    |  6 ++--
 .../hdfs/tools/DFSZKFailoverController.java     | 12 +++----
 .../hdfs/tools/DelegationTokenFetcher.java      |  8 ++---
 .../org/apache/hadoop/hdfs/tools/GetGroups.java |  6 ++--
 .../OfflineEditsBinaryLoader.java               | 10 +++---
 .../offlineImageViewer/FSImageHandler.java      |  7 +++--
 .../tools/offlineImageViewer/FSImageLoader.java |  7 +++--
 .../OfflineImageReconstructor.java              | 10 +++---
 .../offlineImageViewer/OfflineImageViewer.java  |  9 +++---
 .../OfflineImageViewerPB.java                   |  7 +++--
 .../offlineImageViewer/WebImageViewer.java      |  7 +++--
 .../hdfs/util/AtomicFileOutputStream.java       |  6 ++--
 .../hadoop/hdfs/util/LightWeightHashSet.java    |  7 +++--
 .../apache/hadoop/hdfs/util/MD5FileUtils.java   |  8 ++---
 .../hadoop/hdfs/util/PersistentLongFile.java    |  8 ++---
 .../hdfs/web/resources/ExceptionHandler.java    |  7 +++--
 .../apache/hadoop/cli/TestCacheAdminCLI.java    |  7 +++--
 .../hadoop/fs/TestEnhancedByteBufferAccess.java | 10 +++---
 .../hadoop/fs/TestHdfsNativeCodeLoader.java     |  7 +++--
 .../java/org/apache/hadoop/fs/TestUnbuffer.java |  8 ++---
 .../apache/hadoop/hdfs/AdminStatesBaseTest.java |  7 +++--
 .../org/apache/hadoop/hdfs/AppendTestUtil.java  |  6 ++--
 .../apache/hadoop/hdfs/BenchmarkThroughput.java | 13 +++-----
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  6 ++--
 .../org/apache/hadoop/hdfs/FileAppendTest4.java |  7 +++--
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |  9 +++---
 .../hdfs/MiniDFSClusterWithNodeGroup.java       |  7 +++--
 .../hdfs/ReadStripedFileWithDecodingHelper.java |  5 +--
 .../apache/hadoop/hdfs/TestAbandonBlock.java    |  7 +++--
 .../apache/hadoop/hdfs/TestAclsEndToEnd.java    |  8 ++---
 .../hadoop/hdfs/TestAppendSnapshotTruncate.java |  7 +++--
 .../hadoop/hdfs/TestBalancerBandwidth.java      |  7 +++--
 .../hadoop/hdfs/TestBlockMissingException.java  |  7 +++--
 .../hadoop/hdfs/TestClientReportBadBlock.java   |  8 ++---
 .../org/apache/hadoop/hdfs/TestConnCache.java   |  6 ++--
 .../hadoop/hdfs/TestDFSClientFailover.java      |  2 --
 .../hadoop/hdfs/TestDFSClientRetries.java       | 10 +++---
 .../org/apache/hadoop/hdfs/TestDFSFinalize.java |  6 ++--
 .../hdfs/TestDFSInotifyEventInputStream.java    |  6 ++--
 .../apache/hadoop/hdfs/TestDFSPermission.java   |  7 +++--
 .../org/apache/hadoop/hdfs/TestDFSRollback.java |  6 ++--
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |  6 ++--
 .../hadoop/hdfs/TestDFSStartupVersions.java     |  6 ++--
 .../hdfs/TestDFSStorageStateRecovery.java       |  6 ++--
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |  8 ++---
 ...DFSStripedInputStreamWithRandomECPolicy.java |  8 ++---
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |  6 ++--
 ...tputStreamWithFailureWithRandomECPolicy.java |  8 ++---
 ...FSStripedOutputStreamWithRandomECPolicy.java |  8 ++---
 .../org/apache/hadoop/hdfs/TestDFSUpgrade.java  |  7 +++--
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |  7 ++---
 .../org/apache/hadoop/hdfs/TestDataStream.java  |  6 ++--
 .../hadoop/hdfs/TestDataTransferProtocol.java   |  6 ++--
 .../hadoop/hdfs/TestDatanodeRegistration.java   |  7 +++--
 .../apache/hadoop/hdfs/TestDatanodeReport.java  |  6 ++--
 .../hadoop/hdfs/TestDisableConnCache.java       |  6 ++--
 .../hadoop/hdfs/TestEncryptedTransfer.java      | 17 +++++-----
 .../hadoop/hdfs/TestExternalBlockReader.java    |  8 ++---
 .../apache/hadoop/hdfs/TestHDFSServerPorts.java |  7 +++--
 .../org/apache/hadoop/hdfs/TestHDFSTrash.java   |  6 ++--
 .../hdfs/TestInjectionForSimulatedStorage.java  |  6 ++--
 .../org/apache/hadoop/hdfs/TestLargeBlock.java  |  7 +++--
 .../java/org/apache/hadoop/hdfs/TestLease.java  |  6 ++--
 .../apache/hadoop/hdfs/TestLeaseRecovery2.java  |  7 +++--
 .../hadoop/hdfs/TestMissingBlocksAlert.java     |  8 ++---
 .../hadoop/hdfs/TestParallelReadUtil.java       |  6 ++--
 .../org/apache/hadoop/hdfs/TestPipelines.java   |  6 ++--
 .../TestReadStripedFileWithMissingBlocks.java   |  8 ++---
 .../hadoop/hdfs/TestReconstructStripedFile.java |  7 +++--
 .../TestReplaceDatanodeFailureReplication.java  |  8 ++---
 .../hdfs/TestReplaceDatanodeOnFailure.java      |  7 +++--
 .../org/apache/hadoop/hdfs/TestReplication.java |  2 --
 .../apache/hadoop/hdfs/TestRollingUpgrade.java  |  7 +++--
 .../org/apache/hadoop/hdfs/TestSafeMode.java    |  6 ++--
 .../TestUnsetAndChangeDirectoryEcPolicy.java    |  8 ++---
 .../org/apache/hadoop/hdfs/TestWriteRead.java   |  8 ++---
 .../hadoop/hdfs/TestWriteReadStripedFile.java   | 19 ++++++-----
 .../hdfs/TestWriteStripedFileWithFailure.java   |  8 ++---
 .../hadoop/hdfs/net/TestDFSNetworkTopology.java |  8 ++---
 .../hadoop/hdfs/protocol/TestLocatedBlock.java  |  9 +++---
 .../datatransfer/sasl/TestSaslDataTransfer.java |  4 +--
 .../hdfs/qjournal/MiniJournalCluster.java       |  7 +++--
 .../hadoop/hdfs/qjournal/MiniQJMHACluster.java  |  7 +++--
 .../hdfs/qjournal/TestSecureNNWithQJM.java      |  2 --
 .../qjournal/client/TestEpochsAreUnique.java    |  7 +++--
 .../hdfs/qjournal/client/TestQJMWithFaults.java |  6 ++--
 .../client/TestQuorumJournalManager.java        | 16 +++++-----
 .../client/TestQuorumJournalManagerUnit.java    |  4 +--
 .../TestJournalNodeRespectsBindHostKeys.java    |  6 ++--
 .../hdfs/security/TestDelegationToken.java      | 11 ++++---
 .../TestDelegationTokenForProxyUser.java        |  7 +++--
 .../security/token/block/TestBlockToken.java    |  7 +++--
 .../hdfs/server/balancer/TestBalancer.java      | 12 +++----
 .../TestBalancerWithMultipleNameNodes.java      | 12 +++----
 .../balancer/TestBalancerWithNodeGroup.java     |  6 ++--
 .../BaseReplicationPolicyTest.java              |  3 +-
 .../server/blockmanagement/TestBlockInfo.java   |  8 ++---
 .../blockmanagement/TestBlockManager.java       | 12 +++----
 .../TestBlockReportRateLimiting.java            |  7 +++--
 .../TestBlocksWithNotEnoughRacks.java           |  7 +++--
 .../blockmanagement/TestCachedBlocksList.java   |  7 +++--
 .../blockmanagement/TestCorruptReplicaInfo.java |  6 ++--
 .../blockmanagement/TestDatanodeManager.java    |  7 +++--
 .../TestNameNodePrunesMissingStorages.java      |  7 +++--
 .../TestRBWBlockInvalidation.java               |  9 +++---
 .../TestSequentialBlockGroupId.java             |  8 ++---
 .../blockmanagement/TestSequentialBlockId.java  |  7 +++--
 .../server/common/TestGetUriFromString.java     |  7 +++--
 .../hdfs/server/datanode/DataNodeTestUtils.java |  8 ++---
 .../server/datanode/TestBPOfferService.java     |  6 ++--
 .../hdfs/server/datanode/TestBatchIbr.java      | 11 ++++---
 .../TestBlockHasMultipleReplicasOnSameDN.java   |  7 +++--
 .../server/datanode/TestBlockPoolManager.java   |  7 +++--
 .../hdfs/server/datanode/TestBlockRecovery.java | 13 ++++----
 .../server/datanode/TestBlockReplacement.java   |  6 ++--
 .../datanode/TestBpServiceActorScheduler.java   |  7 +++--
 .../TestDataNodeErasureCodingMetrics.java       |  8 ++---
 .../datanode/TestDataNodeFaultInjector.java     |  8 ++---
 .../datanode/TestDataNodeHotSwapVolumes.java    |  8 ++---
 .../datanode/TestDataNodeInitStorage.java       |  7 +++--
 .../server/datanode/TestDataNodeMXBean.java     |  7 +++--
 .../server/datanode/TestDataNodeMetrics.java    |  9 +++---
 .../datanode/TestDataNodeMetricsLogger.java     |  6 ++--
 .../TestDataNodeMultipleRegistrations.java      |  8 ++---
 .../datanode/TestDataNodeReconfiguration.java   |  7 +++--
 .../datanode/TestDataNodeRollingUpgrade.java    |  7 +++--
 .../server/datanode/TestDataNodeTcpNoDelay.java |  8 ++---
 .../TestDataNodeVolumeFailureReporting.java     | 14 ++++-----
 .../datanode/TestDataNodeVolumeMetrics.java     |  8 ++---
 .../TestDatanodeProtocolRetryPolicy.java        | 10 +++---
 .../server/datanode/TestDatanodeRegister.java   |  7 +++--
 .../server/datanode/TestDirectoryScanner.java   |  9 +++---
 ...TestDnRespectsBlockReportSplitThreshold.java |  7 +++--
 .../server/datanode/TestFsDatasetCache.java     | 14 ++++-----
 .../datanode/TestIncrementalBlockReports.java   |  7 +++--
 .../datanode/TestIncrementalBrVariations.java   | 21 +++++++------
 .../datanode/TestReadOnlySharedStorage.java     |  7 +++--
 .../hdfs/server/datanode/TestStorageReport.java |  7 +++--
 .../hdfs/server/datanode/TestTransferRbw.java   |  7 +++--
 .../fsdataset/impl/FsDatasetImplTestUtils.java  |  8 ++---
 .../fsdataset/impl/LazyPersistTestCase.java     |  7 +++--
 .../fsdataset/impl/TestSpaceReservation.java    |  6 ++--
 .../TestDiskBalancerWithMockMover.java          |  7 +++--
 .../hdfs/server/mover/TestStorageMover.java     | 22 ++++++-------
 .../hdfs/server/namenode/FSImageTestUtil.java   | 28 +++++------------
 .../server/namenode/NNThroughputBenchmark.java  |  9 +++---
 .../namenode/OfflineEditsViewerHelper.java      |  8 ++---
 .../hdfs/server/namenode/TestAddBlockRetry.java |  7 +++--
 .../hdfs/server/namenode/TestAllowFormat.java   |  8 ++---
 .../server/namenode/TestAuditLogAtDebug.java    |  6 ++--
 .../hdfs/server/namenode/TestBackupNode.java    | 13 ++++----
 .../server/namenode/TestCacheDirectives.java    |  6 ++--
 .../hdfs/server/namenode/TestCheckpoint.java    |  8 ++---
 .../hdfs/server/namenode/TestClusterId.java     |  9 +++---
 .../hdfs/server/namenode/TestDeadDatanode.java  |  7 +++--
 .../hdfs/server/namenode/TestEditLog.java       | 10 +++---
 .../namenode/TestEditLogFileInputStream.java    |  8 ++---
 .../hdfs/server/namenode/TestEditLogRace.java   | 11 ++++---
 .../hdfs/server/namenode/TestFSDirAttrOp.java   |  7 +++--
 .../hdfs/server/namenode/TestFSDirectory.java   |  7 +++--
 .../server/namenode/TestFSEditLogLoader.java    |  6 ++--
 .../namenode/TestFSImageWithSnapshot.java       |  4 +--
 .../namenode/TestFavoredNodesEndToEnd.java      |  8 ++---
 .../server/namenode/TestFileJournalManager.java | 23 ++++++++------
 .../hdfs/server/namenode/TestFileTruncate.java  | 12 +++----
 .../hadoop/hdfs/server/namenode/TestFsck.java   | 13 ++++----
 .../namenode/TestFsckWithMultipleNameNodes.java |  7 +++--
 .../hdfs/server/namenode/TestHDFSConcat.java    |  7 +++--
 .../hdfs/server/namenode/TestHostsFiles.java    |  8 ++---
 .../hdfs/server/namenode/TestINodeFile.java     |  8 ++---
 .../namenode/TestLargeDirectoryDelete.java      |  9 +++---
 .../hdfs/server/namenode/TestListOpenFiles.java |  9 +++---
 .../TestNNStorageRetentionFunctional.java       |  6 ++--
 .../server/namenode/TestNameEditsConfigs.java   |  6 ++--
 .../namenode/TestNameNodeMetricsLogger.java     |  6 ++--
 .../namenode/TestNameNodeReconfigure.java       |  8 ++---
 .../server/namenode/TestNameNodeRecovery.java   |  9 +++---
 .../TestNameNodeRespectsBindHostKeys.java       |  7 +++--
 .../namenode/TestNameNodeStatusMXBean.java      |  6 ++--
 .../namenode/TestNamenodeCapacityReport.java    |  7 +++--
 .../server/namenode/TestQuotaByStorageType.java | 33 ++++++++++----------
 .../hdfs/server/namenode/TestSaveNamespace.java |  4 +--
 .../hdfs/server/namenode/TestStartup.java       |  7 ++---
 .../server/namenode/TestStorageRestore.java     |  8 ++---
 .../server/namenode/TestStripedINodeFile.java   |  6 ++--
 .../hdfs/server/namenode/ha/HATestUtil.java     |  6 ++--
 .../namenode/ha/TestBootstrapStandby.java       | 13 ++++----
 .../namenode/ha/TestDFSUpgradeWithHA.java       |  7 +++--
 .../namenode/ha/TestDelegationTokensWithHA.java |  8 ++---
 .../server/namenode/ha/TestEditLogTailer.java   |  4 +--
 .../namenode/ha/TestEditLogsDuringFailover.java | 10 +++---
 .../namenode/ha/TestFailureToReadEdits.java     |  8 ++---
 .../hdfs/server/namenode/ha/TestHAFsck.java     |  4 +--
 .../hdfs/server/namenode/ha/TestHAMetrics.java  | 11 ++++---
 .../hdfs/server/namenode/ha/TestHASafeMode.java | 13 ++++----
 .../namenode/ha/TestHAStateTransitions.java     | 12 +++----
 .../namenode/ha/TestInitializeSharedEdits.java  |  7 +++--
 .../namenode/ha/TestRetryCacheWithHA.java       |  7 +++--
 .../namenode/ha/TestSeveralNameNodes.java       |  7 +++--
 .../namenode/ha/TestStandbyBlockManagement.java |  6 ++--
 .../namenode/ha/TestStandbyInProgressTail.java  |  8 ++---
 .../server/namenode/ha/TestStandbyIsHot.java    |  6 ++--
 .../namenode/metrics/TestNameNodeMetrics.java   | 11 ++++---
 ...tINodeFileUnderConstructionWithSnapshot.java |  6 ++--
 .../snapshot/TestOpenFilesWithSnapshot.java     |  8 ++---
 .../snapshot/TestRenameWithSnapshots.java       |  7 +++--
 .../server/namenode/snapshot/TestSnapshot.java  |  4 +--
 .../resources/TestWebHdfsCreatePermissions.java |  7 +++--
 .../web/resources/TestWebHdfsDataLocality.java  |  7 +++--
 .../sps/TestExternalStoragePolicySatisfier.java |  3 +-
 .../shortcircuit/TestShortCircuitCache.java     |  9 +++---
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  |  6 ++--
 .../hadoop/hdfs/tools/TestDFSHAAdmin.java       |  7 +++--
 .../hdfs/tools/TestDFSHAAdminMiniCluster.java   | 13 ++++----
 .../TestOfflineEditsViewer.java                 |  8 ++---
 .../TestOfflineImageViewer.java                 |  9 +++---
 .../TestOfflineImageViewerForAcl.java           |  8 ++---
 ...TestOfflineImageViewerForContentSummary.java |  8 ++---
 .../TestOfflineImageViewerForXAttr.java         |  8 ++---
 .../hdfs/util/TestLightWeightHashSet.java       |  8 ++---
 .../hdfs/util/TestLightWeightLinkedSet.java     | 10 +++---
 .../hdfs/web/TestFSMainOperationsWebHdfs.java   |  4 +--
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java | 10 +++---
 .../hadoop/hdfs/web/TestWebHdfsTimeouts.java    | 18 ++++++-----
 .../web/TestWebHdfsWithMultipleNameNodes.java   | 12 +++----
 .../apache/hadoop/hdfs/web/WebHdfsTestUtil.java |  7 +++--
 .../hadoop/hdfs/web/resources/TestParam.java    |  6 ++--
 .../apache/hadoop/security/TestPermission.java  |  7 +++--
 .../hadoop/security/TestPermissionSymlinks.java |  7 +++--
 318 files changed, 1283 insertions(+), 1173 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
index c3d7071,6302b2a..2380f0d
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
@@@ -23,9 -23,8 +23,9 @@@ import java.util.List
  import java.util.Map;
  import java.util.concurrent.TimeoutException;
  
- import org.apache.commons.logging.Log;
- import org.apache.commons.logging.LogFactory;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
 +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
  import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
  import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
  import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
index ad053a3,ba2b20a..f96fe09
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
@@@ -67,18 -65,8 +67,18 @@@ import com.google.protobuf.TextFormat
   */
  @InterfaceAudience.Private
  public class QuorumJournalManager implements JournalManager {
-   static final Log LOG = LogFactory.getLog(QuorumJournalManager.class);
+   static final Logger LOG = LoggerFactory.getLogger(QuorumJournalManager.class);
  
 +  // This config is not publicly exposed
 +  static final String QJM_RPC_MAX_TXNS_KEY =
 +      "dfs.ha.tail-edits.qjm.rpc.max-txns";
 +  static final int QJM_RPC_MAX_TXNS_DEFAULT = 5000;
 +
 +  // Maximum number of transactions to fetch at a time when using the
 +  // RPC edit fetch mechanism
 +  private final int maxTxnsPerRpc;
 +  // Whether or not in-progress tailing is enabled in the configuration
 +  private final boolean inProgressTailingEnabled;
    // Timeouts for which the QJM will wait for each of the following actions.
    private final int startSegmentTimeoutMs;
    private final int prepareRecoveryTimeoutMs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
index ccf8ba2,39afabc..9e204cb
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
@@@ -35,11 -32,10 +35,11 @@@ import java.util.List
  import java.util.concurrent.TimeUnit;
  
  import org.apache.commons.lang3.Range;
- import org.apache.commons.logging.Log;
- import org.apache.commons.logging.LogFactory;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.fs.FileUtil;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
  import org.apache.hadoop.hdfs.qjournal.protocol.JournalNotFormattedException;
  import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
  import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournaledEditsCache.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournaledEditsCache.java
index 2693301,0000000..387caa1
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournaledEditsCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournaledEditsCache.java
@@@ -1,412 -1,0 +1,412 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hdfs.qjournal.server;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import java.io.ByteArrayInputStream;
 +import java.io.ByteArrayOutputStream;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.NavigableMap;
 +import java.util.TreeMap;
 +import java.util.concurrent.locks.ReadWriteLock;
 +import java.util.concurrent.locks.ReentrantReadWriteLock;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 +import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader;
 +import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 +import org.apache.hadoop.util.AutoCloseableLock;
 +
 +/**
 + * An in-memory cache of edits in their serialized form. This is used to serve
 + * the {@link Journal#getJournaledEdits(long, int)} call, used by the
 + * QJM when {@value DFSConfigKeys#DFS_HA_TAILEDITS_INPROGRESS_KEY} is
 + * enabled.
 + *
 + * <p>When a batch of edits is received by the JournalNode, it is put into this
 + * cache via {@link #storeEdits(byte[], long, long, int)}. Edits must be
 + * stored contiguously; if a batch of edits is stored that does not align with
 + * the previously stored edits, the cache will be cleared before storing new
 + * edits to avoid gaps. This decision is made because gaps are only handled
 + * when in recovery mode, which the cache is not intended to be used for.
 + *
 + * <p>Batches of edits are stored in a {@link TreeMap} mapping the starting
 + * transaction ID of the batch to the data buffer. Upon retrieval, the
 + * relevant data buffers are concatenated together and a header is added
 + * to construct a fully-formed edit data stream.
 + *
 + * <p>The cache is of a limited size capacity determined by
 + * {@value DFSConfigKeys#DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY}. If the capacity
 + * is exceeded after adding a new batch of edits, batches of edits are removed
 + * until the total size is less than the capacity, starting from the ones
 + * containing the oldest transactions. Transactions range in size, but a
 + * decent rule of thumb is that 200 bytes are needed per transaction. Monitoring
 + * the {@link JournalMetrics#rpcRequestCacheMissAmount} metric is recommended
 + * to determine if the cache is too small; it will indicate both how many
 + * cache misses occurred, and how many more transactions would have been
 + * needed in the cache to serve the request.
 + */
 +class JournaledEditsCache {
 +
 +  private static final int INVALID_LAYOUT_VERSION = 0;
 +  private static final long INVALID_TXN_ID = -1;
 +
 +  /** The capacity, in bytes, of this cache. */
 +  private final int capacity;
 +
 +  /**
 +   * Read/write lock pair wrapped in AutoCloseable; these refer to the same
 +   * underlying lock.
 +   */
 +  private final AutoCloseableLock readLock;
 +  private final AutoCloseableLock writeLock;
 +
 +  // ** Start lock-protected fields **
 +
 +  /**
 +   * Stores the actual data as a mapping of the StartTxnId of a batch of edits
 +   * to the serialized batch of edits. Stores only contiguous ranges; that is,
 +   * the last transaction ID in one batch is always one less than the first
 +   * transaction ID in the next batch. Though the map is protected by the lock,
 +   * individual data buffers are immutable and can be accessed without locking.
 +   */
 +  private final NavigableMap<Long, byte[]> dataMap = new TreeMap<>();
 +  /** Stores the layout version currently present in the cache. */
 +  private int layoutVersion = INVALID_LAYOUT_VERSION;
 +  /** Stores the serialized version of the header for the current version. */
 +  private ByteBuffer layoutHeader;
 +
 +  /**
 +   * The lowest/highest transaction IDs present in the cache.
 +   * {@value INVALID_TXN_ID} if there are no transactions in the cache.
 +   */
 +  private long lowestTxnId;
 +  private long highestTxnId;
 +  /**
 +   * The lowest transaction ID that was ever present in the cache since last
 +   * being reset (i.e. since initialization or since reset due to being out of
 +   * sync with the Journal). Until the cache size goes above capacity, this is
 +   * equal to lowestTxnId.
 +   */
 +  private long initialTxnId;
 +  /** The current total size of all buffers in this cache. */
 +  private int totalSize;
 +
 +  // ** End lock-protected fields **
 +
 +  JournaledEditsCache(Configuration conf) {
 +    capacity = conf.getInt(DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY,
 +        DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_DEFAULT);
 +    if (capacity > 0.9 * Runtime.getRuntime().maxMemory()) {
 +      Journal.LOG.warn(String.format("Cache capacity is set at %d bytes but " +
 +          "maximum JVM memory is only %d bytes. It is recommended that you " +
 +          "decrease the cache size or increase the heap size.",
 +          capacity, Runtime.getRuntime().maxMemory()));
 +    }
 +    Journal.LOG.info("Enabling the journaled edits cache with a capacity " +
 +        "of bytes: " + capacity);
 +    ReadWriteLock lock = new ReentrantReadWriteLock(true);
 +    readLock = new AutoCloseableLock(lock.readLock());
 +    writeLock = new AutoCloseableLock(lock.writeLock());
 +    initialize(INVALID_TXN_ID);
 +  }
 +
 +  /**
 +   * Fetch the data for edits starting at the specific transaction ID, fetching
 +   * up to {@code maxTxns} transactions. Populates a list of output buffers
 +   * which contains a serialized version of the edits, and returns the count of
 +   * edits contained within the serialized buffers. The serialized edits are
 +   * prefixed with a standard edit log header containing information about the
 +   * layout version. The transactions returned are guaranteed to have contiguous
 +   * transaction IDs.
 +   *
 +   * If {@code requestedStartTxn} is higher than the highest transaction which
 +   * has been added to this cache, a response with an empty buffer and a
 +   * transaction count of 0 will be returned. If {@code requestedStartTxn} is
 +   * lower than the lowest transaction currently contained in this cache, or no
 +   * transactions have yet been added to the cache, an exception will be thrown.
 +   *
 +   * @param requestedStartTxn The ID of the first transaction to return. If any
 +   *                          transactions are returned, it is guaranteed that
 +   *                          the first one will have this ID.
 +   * @param maxTxns The maximum number of transactions to return.
 +   * @param outputBuffers A list to populate with output buffers. When
 +   *                      concatenated, these form a full response.
 +   * @return The number of transactions contained within the set of output
 +   *         buffers.
 +   * @throws IOException If transactions are requested which cannot be served
 +   *                     by this cache.
 +   */
 +  int retrieveEdits(long requestedStartTxn, int maxTxns,
 +      List<ByteBuffer> outputBuffers) throws IOException {
 +    int txnCount = 0;
 +
 +    try (AutoCloseableLock l = readLock.acquire()) {
 +      if (lowestTxnId == INVALID_TXN_ID || requestedStartTxn < lowestTxnId) {
 +        throw getCacheMissException(requestedStartTxn);
 +      } else if (requestedStartTxn > highestTxnId) {
 +        return 0;
 +      }
 +      outputBuffers.add(layoutHeader);
 +      Iterator<Map.Entry<Long, byte[]>> incrBuffIter =
 +          dataMap.tailMap(dataMap.floorKey(requestedStartTxn), true)
 +              .entrySet().iterator();
 +      long prevTxn = requestedStartTxn;
 +      byte[] prevBuf = null;
 +      // Stop when maximum transactions reached...
 +      while ((txnCount < maxTxns) &&
 +          // ... or there are no more entries ...
 +          (incrBuffIter.hasNext() || prevBuf != null)) {
 +        long currTxn;
 +        byte[] currBuf;
 +        if (incrBuffIter.hasNext()) {
 +          Map.Entry<Long, byte[]> ent = incrBuffIter.next();
 +          currTxn = ent.getKey();
 +          currBuf = ent.getValue();
 +        } else {
 +          // This accounts for the trailing entry
 +          currTxn = highestTxnId + 1;
 +          currBuf = null;
 +        }
 +        if (prevBuf != null) { // True except for the first loop iteration
 +          outputBuffers.add(ByteBuffer.wrap(prevBuf));
 +          // if prevTxn < requestedStartTxn, the extra transactions will get
 +          // removed after the loop, so don't include them in the txn count
 +          txnCount += currTxn - Math.max(requestedStartTxn, prevTxn);
 +        }
 +        prevTxn = currTxn;
 +        prevBuf = currBuf;
 +      }
 +      // Release the lock before doing operations on the buffers (deserializing
 +      // to find transaction boundaries, and copying into an output buffer)
 +    }
 +    // Remove extra leading transactions in the first buffer
 +    ByteBuffer firstBuf = outputBuffers.get(1); // 0th is the header
 +    firstBuf.position(
 +        findTransactionPosition(firstBuf.array(), requestedStartTxn));
 +    // Remove trailing transactions in the last buffer if necessary
 +    if (txnCount > maxTxns) {
 +      ByteBuffer lastBuf = outputBuffers.get(outputBuffers.size() - 1);
 +      int limit =
 +          findTransactionPosition(lastBuf.array(), requestedStartTxn + maxTxns);
 +      lastBuf.limit(limit);
 +      txnCount = maxTxns;
 +    }
 +
 +    return txnCount;
 +  }
 +
 +  /**
 +   * Store a batch of serialized edits into this cache. Removes old batches
 +   * as necessary to keep the total size of the cache below the capacity.
 +   * See the class Javadoc for more info.
 +   *
 +   * This attempts to always handle malformed inputs gracefully rather than
 +   * throwing an exception, to allow the rest of the Journal's operations
 +   * to proceed normally.
 +   *
 +   * @param inputData A buffer containing edits in serialized form
 +   * @param newStartTxn The txn ID of the first edit in {@code inputData}
 +   * @param newEndTxn The txn ID of the last edit in {@code inputData}
 +   * @param newLayoutVersion The version of the layout used to serialize
 +   *                         the edits
 +   */
 +  void storeEdits(byte[] inputData, long newStartTxn, long newEndTxn,
 +      int newLayoutVersion) {
 +    if (newStartTxn < 0 || newEndTxn < newStartTxn) {
 +      Journal.LOG.error(String.format("Attempted to cache data of length %d " +
 +          "with newStartTxn %d and newEndTxn %d",
 +          inputData.length, newStartTxn, newEndTxn));
 +      return;
 +    }
 +    try (AutoCloseableLock l = writeLock.acquire()) {
 +      if (newLayoutVersion != layoutVersion) {
 +        try {
 +          updateLayoutVersion(newLayoutVersion, newStartTxn);
 +        } catch (IOException ioe) {
 +          Journal.LOG.error(String.format("Unable to save new edits [%d, %d] " +
 +              "due to exception when updating to new layout version %d",
 +              newStartTxn, newEndTxn, newLayoutVersion), ioe);
 +          return;
 +        }
 +      } else if (lowestTxnId == INVALID_TXN_ID) {
 +        Journal.LOG.info("Initializing edits cache starting from txn ID " +
 +            newStartTxn);
 +        initialize(newStartTxn);
 +      } else if (highestTxnId + 1 != newStartTxn) {
 +        // Cache is out of sync; clear to avoid storing noncontiguous regions
 +        Journal.LOG.error(String.format("Edits cache is out of sync; " +
 +            "looked for next txn id at %d but got start txn id for " +
 +            "cache put request at %d. Reinitializing at new request.",
 +            highestTxnId + 1, newStartTxn));
 +        initialize(newStartTxn);
 +      }
 +
 +      while ((totalSize + inputData.length) > capacity && !dataMap.isEmpty()) {
 +        Map.Entry<Long, byte[]> lowest = dataMap.firstEntry();
 +        dataMap.remove(lowest.getKey());
 +        totalSize -= lowest.getValue().length;
 +      }
 +      if (inputData.length > capacity) {
 +        initialize(INVALID_TXN_ID);
 +        Journal.LOG.warn(String.format("A single batch of edits was too " +
 +                "large to fit into the cache: startTxn = %d, endTxn = %d, " +
 +                "input length = %d. The capacity of the cache (%s) must be " +
 +                "increased for it to work properly (current capacity %d)." +
 +                "Cache is now empty.",
 +            newStartTxn, newEndTxn, inputData.length,
 +            DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, capacity));
 +        return;
 +      }
 +      if (dataMap.isEmpty()) {
 +        lowestTxnId = newStartTxn;
 +      } else {
 +        lowestTxnId = dataMap.firstKey();
 +      }
 +
 +      dataMap.put(newStartTxn, inputData);
 +      highestTxnId = newEndTxn;
 +      totalSize += inputData.length;
 +    }
 +  }
 +
 +  /**
 +   * Skip through a given stream of edits until the given transaction ID is
 +   * found. Return the number of bytes that appear prior to the given
 +   * transaction.
 +   *
 +   * @param buf A buffer containing a stream of serialized edits
 +   * @param txnId The transaction ID to search for
 +   * @return The number of bytes appearing in {@code buf} <i>before</i>
 +   *         the start of the transaction with ID {@code txnId}.
 +   */
 +  private int findTransactionPosition(byte[] buf, long txnId)
 +      throws IOException {
 +    ByteArrayInputStream bais = new ByteArrayInputStream(buf);
 +    FSEditLogLoader.PositionTrackingInputStream tracker =
 +        new FSEditLogLoader.PositionTrackingInputStream(bais);
 +    FSEditLogOp.Reader reader = FSEditLogOp.Reader.create(
 +        new DataInputStream(tracker), tracker, layoutVersion);
 +    long previousPos = 0;
 +    while (reader.scanOp() < txnId) {
 +      previousPos = tracker.getPos();
 +    }
 +    // tracker is backed by a byte[]; position cannot go above an integer
 +    return (int) previousPos;
 +  }
 +
 +  /**
 +   * Update the layout version of the cache. This clears out all existing
 +   * entries, and populates the new layout version and header for that version.
 +   *
 +   * @param newLayoutVersion The new layout version to be stored in the cache
 +   * @param newStartTxn The new lowest transaction in the cache
 +   */
 +  private void updateLayoutVersion(int newLayoutVersion, long newStartTxn)
 +      throws IOException {
 +    StringBuilder logMsg = new StringBuilder()
 +        .append("Updating edits cache to use layout version ")
 +        .append(newLayoutVersion)
 +        .append(" starting from txn ID ")
 +        .append(newStartTxn);
 +    if (layoutVersion != INVALID_LAYOUT_VERSION) {
 +      logMsg.append("; previous version was ").append(layoutVersion)
 +          .append("; old entries will be cleared.");
 +    }
-     Journal.LOG.info(logMsg);
++    Journal.LOG.info(logMsg.toString());
 +    initialize(newStartTxn);
 +    ByteArrayOutputStream baos = new ByteArrayOutputStream();
 +    EditLogFileOutputStream.writeHeader(newLayoutVersion,
 +        new DataOutputStream(baos));
 +    layoutVersion = newLayoutVersion;
 +    layoutHeader = ByteBuffer.wrap(baos.toByteArray());
 +  }
 +
 +  /**
 +   * Initialize the cache back to a clear state.
 +   *
 +   * @param newInitialTxnId The new lowest transaction ID stored in the cache.
 +   *                        This should be {@value INVALID_TXN_ID} if the cache
 +   *                        is to remain empty at this time.
 +   */
 +  private void initialize(long newInitialTxnId) {
 +    dataMap.clear();
 +    totalSize = 0;
 +    initialTxnId = newInitialTxnId;
 +    lowestTxnId = initialTxnId;
 +    highestTxnId = INVALID_TXN_ID; // this will be set later
 +  }
 +
 +  /**
 +   * Return the underlying data buffer used to store information about the
 +   * given transaction ID.
 +   *
 +   * @param txnId Transaction ID whose containing buffer should be fetched.
 +   * @return The data buffer for the transaction
 +   */
 +  @VisibleForTesting
 +  byte[] getRawDataForTests(long txnId) {
 +    try (AutoCloseableLock l = readLock.acquire()) {
 +      return dataMap.floorEntry(txnId).getValue();
 +    }
 +  }
 +
 +  private CacheMissException getCacheMissException(long requestedTxnId) {
 +    if (lowestTxnId == INVALID_TXN_ID) {
 +      return new CacheMissException(0, "Cache is empty; either it was never " +
 +          "written to or the last write overflowed the cache capacity.");
 +    } else if (requestedTxnId < initialTxnId) {
 +      return new CacheMissException(initialTxnId - requestedTxnId,
 +          "Cache started at txn ID %d but requested txns starting at %d.",
 +          initialTxnId, requestedTxnId);
 +    } else {
 +      return new CacheMissException(lowestTxnId - requestedTxnId,
 +          "Oldest txn ID available in the cache is %d, but requested txns " +
 +              "starting at %d. The cache size (%s) may need to be increased " +
 +              "to hold more transactions (currently %d bytes containing %d " +
 +              "transactions)", lowestTxnId, requestedTxnId,
 +          DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, capacity,
 +          highestTxnId - lowestTxnId + 1);
 +    }
 +  }
 +
 +  static class CacheMissException extends IOException {
 +
 +    private static final long serialVersionUID = 0L;
 +
 +    private final long cacheMissAmount;
 +
 +    CacheMissException(long cacheMissAmount, String msgFormat,
 +        Object... msgArgs) {
 +      super(String.format(msgFormat, msgArgs));
 +      this.cacheMissAmount = cacheMissAmount;
 +    }
 +
 +    long getCacheMissAmount() {
 +      return cacheMissAmount;
 +    }
 +
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
index f533bc7,f7c3a27..f3bb954
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
@@@ -40,11 -40,10 +40,11 @@@ import java.util.List
  import java.util.concurrent.ExecutorService;
  import java.util.concurrent.TimeoutException;
  
- import org.apache.commons.logging.Log;
- import org.apache.commons.logging.LogFactory;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
  import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
  import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
  import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
index 0843bfe,3eca80f..3824b83
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
@@@ -32,9 -32,8 +32,9 @@@ import java.net.HttpURLConnection
  import java.net.URL;
  import java.util.EnumMap;
  
 +import com.google.protobuf.ByteString;
- import org.apache.commons.logging.Log;
- import org.apache.commons.logging.LogFactory;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.fs.permission.FsPermission;
  import org.apache.hadoop.fs.permission.PermissionStatus;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
index 0420579,2bdada4..8394073
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
@@@ -22,14 -21,12 +22,14 @@@ import static org.junit.Assert.assertNo
  import static org.junit.Assert.assertNull;
  
  import java.io.File;
 +import java.io.FilenameFilter;
  import java.io.IOException;
  import java.net.URI;
 +import java.util.Iterator;
  import java.util.List;
  
- import org.apache.commons.logging.Log;
- import org.apache.commons.logging.LogFactory;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.fs.permission.FsPermission;
  import org.apache.hadoop.hdfs.DFSConfigKeys;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9ade422b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org