Skip to content

Conversation

@jianghuazhu
Copy link
Contributor

@jianghuazhu jianghuazhu commented Sep 10, 2021

Description of PR

INodeMap#NAMESPACE_KEY_DEPTH&NUM_RANGES_STATIC is configurable.

How was this patch tested?

INodeMap#NAMESPACE_KEY_DEPTH and NUM_RANGES_STATIC can be successfully obtained after changing the values of INodeMap#NAMESPACE_KEY_DEPTH and NUM_RANGES_STATIC in the Configuration.

jira:HDFS-16220

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 1m 14s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 1 new or modified test files.
_ fgl Compile Tests _
+1 💚 mvninstall 30m 53s fgl passed
+1 💚 compile 1m 24s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 compile 1m 17s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 checkstyle 1m 3s fgl passed
+1 💚 mvnsite 1m 26s fgl passed
+1 💚 javadoc 0m 56s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 1m 27s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 spotbugs 3m 9s fgl passed
+1 💚 shadedclient 16m 28s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 13s the patch passed
+1 💚 compile 1m 13s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javac 1m 13s the patch passed
+1 💚 compile 1m 10s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 javac 1m 10s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 53s /results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs-project/hadoop-hdfs: The patch generated 10 new + 285 unchanged - 0 fixed = 295 total (was 285)
+1 💚 mvnsite 1m 15s the patch passed
-1 ❌ xml 0m 2s /xml.txt The patch has 1 ill-formed XML file(s).
+1 💚 javadoc 0m 46s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 1m 21s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ spotbugs 3m 13s /new-spotbugs-hadoop-hdfs-project_hadoop-hdfs.html hadoop-hdfs-project/hadoop-hdfs generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)
-1 ❌ shadedclient 15m 51s patch has errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 15m 15s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 35s The patch does not generate ASF License warnings.
100m 44s
Reason Tests
XML Parsing Error(s):
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
SpotBugs module:hadoop-hdfs-project/hadoop-hdfs
Write to static field org.apache.hadoop.hdfs.server.namenode.INodeMap.namespaceKeyDepth from instance method new org.apache.hadoop.hdfs.server.namenode.INodeMap(INodeDirectory, FSNamesystem, Configuration) At INodeMap.java:from instance method new org.apache.hadoop.hdfs.server.namenode.INodeMap(INodeDirectory, FSNamesystem, Configuration) At INodeMap.java:[line 206]
Failed junit tests hadoop.hdfs.server.blockmanagement.TestBlocksWithNotEnoughRacks
hadoop.hdfs.server.namenode.TestFsckWithMultipleNameNodes
hadoop.hdfs.TestSafeModeWithStripedFile
hadoop.hdfs.TestPread
hadoop.hdfs.server.datanode.TestDatanodeRegister
hadoop.hdfs.server.datanode.TestLargeBlockReport
hadoop.hdfs.server.namenode.TestINodeMap
hadoop.hdfs.server.namenode.ha.TestInitializeSharedEdits
hadoop.hdfs.server.datanode.TestDataNodeFSDataSetSink
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyPersistReplicaRecovery
hadoop.hdfs.server.namenode.TestQuotaWithStripedBlocksWithRandomECPolicy
hadoop.hdfs.server.namenode.snapshot.TestSetQuotaWithSnapshot
hadoop.hdfs.TestDFSInputStreamBlockLocations
hadoop.hdfs.server.namenode.TestLeaseManager
hadoop.hdfs.qjournal.TestSecureNNWithQJM
hadoop.hdfs.server.blockmanagement.TestPendingInvalidateBlock
hadoop.hdfs.TestFsShellPermission
hadoop.hdfs.TestEncryptedTransfer
hadoop.hdfs.TestLeaseRecoveryStriped
hadoop.hdfs.qjournal.server.TestJournalNodeMXBean
hadoop.hdfs.server.datanode.TestDataNodeFaultInjector
hadoop.hdfs.server.datanode.TestSimulatedFSDatasetWithMultipleStorages
hadoop.hdfs.server.namenode.snapshot.TestGetContentSummaryWithSnapshot
hadoop.hdfs.TestErasureCodingPolicyWithSnapshotWithRandomECPolicy
hadoop.fs.viewfs.TestViewFsLinkFallback
hadoop.hdfs.server.namenode.TestNameNodeRespectsBindHostKeys
hadoop.hdfs.TestDataTransferProtocol
hadoop.hdfs.TestListFilesInFileContext
hadoop.hdfs.server.blockmanagement.TestPendingDataNodeMessages
hadoop.hdfs.server.namenode.TestAddStripedBlockInFBR
hadoop.hdfs.qjournal.client.TestQuorumJournalManager
hadoop.hdfs.server.namenode.ha.TestStandbyBlockManagement
hadoop.hdfs.server.namenode.TestAuditLogAtDebug
hadoop.hdfs.web.TestWebHdfsTokens
hadoop.hdfs.server.namenode.snapshot.TestFileContextSnapshot
hadoop.hdfs.server.blockmanagement.TestPendingReconstruction
hadoop.hdfs.TestClose
hadoop.hdfs.TestByteBufferPread
hadoop.fs.contract.hdfs.TestHDFSContractSetTimes
hadoop.metrics2.sink.TestRollingFileSystemSinkWithHdfs
hadoop.hdfs.server.namenode.ha.TestStandbyIsHot
hadoop.hdfs.TestFileChecksum
hadoop.hdfs.TestFileConcurrentReader
hadoop.fs.TestSymlinkHdfsDisable
hadoop.hdfs.server.namenode.TestStripedINodeFile
hadoop.hdfs.server.datanode.TestDiskError
hadoop.hdfs.TestErasureCodeBenchmarkThroughput
hadoop.hdfs.protocol.datatransfer.TestPacketReceiver
hadoop.hdfs.TestAbandonBlock
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDiffReport
hadoop.hdfs.server.namenode.TestQuotaWithStripedBlocks
hadoop.hdfs.TestAppendSnapshotTruncate
hadoop.fs.viewfs.TestViewFileSystemAtHdfsRoot
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewerWithStripedBlocks
hadoop.hdfs.TestIsMethodSupported
hadoop.hdfs.server.datanode.fsdataset.impl.TestSpaceReservation
hadoop.hdfs.server.blockmanagement.TestSlowPeerTracker
hadoop.hdfs.TestDeprecatedKeys
hadoop.hdfs.server.namenode.ha.TestBootstrapAliasmap
hadoop.hdfs.TestApplyingStoragePolicy
hadoop.fs.viewfs.TestViewFileSystemWithXAttrs
hadoop.hdfs.server.namenode.ha.TestRemoteNameNodeInfo
hadoop.hdfs.server.namenode.TestFSImageWithAcl
hadoop.hdfs.TestFileAppendRestart
hadoop.hdfs.server.namenode.TestFSImage
hadoop.hdfs.server.namenode.snapshot.TestOpenFilesWithSnapshot
hadoop.fs.contract.hdfs.TestHDFSContractConcat
hadoop.hdfs.server.datanode.TestDataNodeReconfiguration
hadoop.TestGenericRefresh
hadoop.hdfs.TestFileCreation
hadoop.hdfs.server.namenode.ha.TestDFSUpgradeWithHA
hadoop.hdfs.qjournal.server.TestJournalNodeSync
hadoop.hdfs.TestParallelRead
hadoop.hdfs.client.impl.TestBlockReaderFactory
hadoop.hdfs.server.blockmanagement.TestHeartbeatHandling
hadoop.hdfs.server.namenode.ha.TestLossyRetryInvocationHandler
hadoop.hdfs.server.datanode.TestDataNodeMetricsLogger
hadoop.hdfs.security.token.block.TestBlockToken
hadoop.hdfs.server.namenode.snapshot.TestSnapshotRename
hadoop.fs.TestFcHdfsPermission
hadoop.hdfs.server.blockmanagement.TestNodeCount
hadoop.hdfs.server.namenode.ha.TestHASafeMode
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewerForStoragePolicy
hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletionGc
hadoop.hdfs.TestFileAppend2
hadoop.net.TestNetworkTopology
hadoop.hdfs.server.namenode.ha.TestHAStateTransitions
hadoop.hdfs.server.namenode.snapshot.TestSnapshotMetrics
hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfierWithStripedFile
hadoop.hdfs.server.namenode.TestNameNodeRetryCacheMetrics
hadoop.hdfs.TestSetTimes
hadoop.hdfs.server.namenode.TestListCorruptFileBlocks
hadoop.hdfs.server.namenode.TestNNStorageRetentionFunctional
hadoop.fs.TestGlobPaths
hadoop.hdfs.server.blockmanagement.TestSequentialBlockGroupId
hadoop.hdfs.server.datanode.TestDataNodeHotSwapVolumes
hadoop.hdfs.server.namenode.TestNameNodeAcl
hadoop.hdfs.server.namenode.snapshot.TestRenameWithOrderedSnapshotDeletion
hadoop.hdfs.server.datanode.TestDataDirs
hadoop.hdfs.server.namenode.TestUpgradeDomainBlockPlacementPolicy
hadoop.hdfs.TestDFSStripedOutputStreamWithFailureWithRandomECPolicy
hadoop.hdfs.TestFileLengthOnClusterRestart
hadoop.tools.TestTools
hadoop.hdfs.server.datanode.TestBlockHasMultipleReplicasOnSameDN
hadoop.hdfs.server.namenode.TestSecureNameNode
hadoop.hdfs.server.namenode.TestFSNamesystemLock
hadoop.hdfs.server.blockmanagement.TestReplicationPolicyWithNodeGroup
hadoop.hdfs.server.namenode.TestEditLogJournalFailures
hadoop.hdfs.server.namenode.TestReconstructStripedBlocks
hadoop.hdfs.TestMaintenanceState
hadoop.hdfs.server.namenode.TestAddBlockRetry
hadoop.cli.TestAclCLIWithPosixAclInheritance
hadoop.metrics2.sink.TestRollingFileSystemSinkWithSecureHdfs
hadoop.hdfs.server.namenode.TestNetworkTopologyServlet
hadoop.hdfs.server.datanode.TestStorageReport
hadoop.hdfs.TestErasureCodingPolicyWithSnapshot
hadoop.fs.viewfs.TestViewFsHdfs
hadoop.hdfs.server.namenode.TestNameNodeMetadataConsistency
hadoop.hdfs.TestDatanodeStartupFixesLegacyStorageIDs
hadoop.hdfs.TestFetchImage
hadoop.hdfs.server.datanode.TestBlockReplacement
hadoop.hdfs.qjournal.TestNNWithQJM
hadoop.hdfs.server.namenode.ha.TestDNFencingWithReplication
hadoop.hdfs.server.namenode.ha.TestStateTransitionFailure
hadoop.hdfs.TestInjectionForSimulatedStorage
hadoop.hdfs.server.datanode.TestDataNodeMetrics
hadoop.hdfs.server.namenode.TestDefaultBlockPlacementPolicy
hadoop.hdfs.server.namenode.TestBlockUnderConstruction
hadoop.hdfs.TestDFSInotifyEventInputStream
hadoop.hdfs.TestDecommissionWithStriped
hadoop.hdfs.server.namenode.TestAddOverReplicatedStripedBlocks
hadoop.hdfs.server.datanode.TestDataNodePeerMetrics
hadoop.hdfs.TestViewDistributedFileSystemWithMountLinks
hadoop.cli.TestDeleteCLI
hadoop.hdfs.server.namenode.TestNameNodeMXBean
hadoop.hdfs.server.datanode.TestDataNodeVolumeFailureToleration
hadoop.hdfs.server.namenode.TestFileContextXAttr
hadoop.hdfs.TestDistributedFileSystem
hadoop.cli.TestHDFSCLI
hadoop.hdfs.TestSafeMode
hadoop.fs.contract.hdfs.TestHDFSContractMkdir
hadoop.hdfs.server.namenode.ha.TestBootstrapStandbyWithInProgressTailing
hadoop.hdfs.TestDataStream
hadoop.hdfs.server.datanode.TestTriggerBlockReport
hadoop.hdfs.TestFileCreationEmpty
hadoop.hdfs.server.datanode.TestDnRespectsBlockReportSplitThreshold
hadoop.hdfs.server.namenode.TestCheckpoint
hadoop.hdfs.server.namenode.snapshot.TestXAttrWithSnapshot
hadoop.hdfs.server.namenode.TestStartup
hadoop.hdfs.server.namenode.TestGetBlockLocations
hadoop.hdfs.server.blockmanagement.TestComputeInvalidateWork
hadoop.hdfs.TestDFSUtil
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyPersistFiles
hadoop.hdfs.TestExternalBlockReader
hadoop.hdfs.server.namenode.TestFSImageWithSnapshot
hadoop.hdfs.TestDFSRename
hadoop.fs.viewfs.TestViewFileSystemHdfs
hadoop.hdfs.TestRollingUpgradeRollback
hadoop.hdfs.server.datanode.TestDataNodeUUID
hadoop.hdfs.server.namenode.ha.TestHAFsck
hadoop.hdfs.server.namenode.ha.TestDelegationTokensWithHA
hadoop.hdfs.server.datanode.TestDataNodeExit
hadoop.hdfs.server.namenode.TestEditLogAutoroll
hadoop.hdfs.TestFileCreationDelete
hadoop.hdfs.TestStateAlignmentContextWithHA
hadoop.hdfs.server.diskbalancer.TestDiskBalancerWithMockMover
hadoop.hdfs.TestDatanodeConfig
hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeWithHdfsScheme
hadoop.hdfs.server.namenode.TestAuditLogs
hadoop.hdfs.TestParallelShortCircuitRead
hadoop.fs.TestEnhancedByteBufferAccess
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewerForAcl
hadoop.hdfs.server.namenode.TestAuditLoggerWithCommands
hadoop.hdfs.TestUnsetAndChangeDirectoryEcPolicy
hadoop.hdfs.TestViewDistributedFileSystemContract
hadoop.hdfs.TestPersistBlocks
hadoop.hdfs.TestDFSStripedInputStreamWithRandomECPolicy
hadoop.hdfs.TestDFSStartupVersions
hadoop.hdfs.tools.TestDFSZKFailoverController
hadoop.hdfs.server.diskbalancer.command.TestDiskBalancerCommand
hadoop.hdfs.server.namenode.snapshot.TestDiffListBySkipList
hadoop.hdfs.server.namenode.TestFavoredNodesEndToEnd
hadoop.hdfs.server.datanode.TestDatanodeStartupOptions
hadoop.hdfs.server.datanode.fsdataset.impl.TestScrLazyPersistFiles
hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeHdfsFileSystemContract
hadoop.hdfs.qjournal.client.TestQJMWithFaults
hadoop.fs.TestFcHdfsSetUMask
hadoop.hdfs.TestDeadNodeDetection
hadoop.hdfs.server.namenode.ha.TestSeveralNameNodes
hadoop.hdfs.server.namenode.ha.TestFailureToReadEdits
hadoop.hdfs.server.namenode.TestHostsFiles
hadoop.hdfs.server.datanode.TestRefreshNamenodes
hadoop.hdfs.server.datanode.TestDataNodeVolumeFailureReporting
hadoop.hdfs.TestDecommissionWithStripedBackoffMonitor
hadoop.hdfs.server.blockmanagement.TestRBWBlockInvalidation
hadoop.hdfs.server.namenode.TestNamenodeStorageDirectives
hadoop.hdfs.TestDFSShellGenericOptions
hadoop.hdfs.server.namenode.TestStartupOptionUpgrade
hadoop.hdfs.TestRollingUpgradeDowngrade
hadoop.hdfs.server.balancer.TestBalancer
hadoop.hdfs.server.namenode.ha.TestBootstrapStandbyWithQJM
hadoop.hdfs.TestDFSClientFailover
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyWriter
hadoop.hdfs.server.namenode.TestFsck
hadoop.hdfs.server.diskbalancer.TestConnectors
hadoop.hdfs.server.blockmanagement.TestSortLocatedBlock
hadoop.hdfs.server.namenode.snapshot.TestSnapshotNameWithInvalidCharacters
hadoop.hdfs.server.namenode.TestCommitBlockWithInvalidGenStamp
hadoop.hdfs.tools.TestStoragePolicyCommands
hadoop.hdfs.tools.TestViewFileSystemOverloadSchemeWithFSCommands
hadoop.hdfs.server.blockmanagement.TestNameNodePrunesMissingStorages
hadoop.tools.TestJMXGet
hadoop.fs.TestSWebHdfsFileContextMainOperations
hadoop.hdfs.server.balancer.TestKeyManager
hadoop.fs.viewfs.TestViewFileSystemWithTruncate
hadoop.hdfs.server.balancer.TestBalancerRPCDelay
hadoop.hdfs.security.TestDelegationTokenForProxyUser
hadoop.hdfs.server.datanode.metrics.TestDataNodeOutlierDetectionViaMetrics
hadoop.fs.TestSymlinkHdfsFileContext
hadoop.hdfs.server.namenode.web.resources.TestWebHdfsDataLocality
hadoop.hdfs.tools.TestDelegationTokenFetcher
hadoop.hdfs.server.namenode.TestNamenodeRetryCache
hadoop.hdfs.server.namenode.ha.TestHAMetrics
hadoop.hdfs.TestSetrepDecreasing
hadoop.hdfs.TestStoragePolicyPermissionSettings
hadoop.hdfs.server.balancer.TestBalancerWithMultipleNameNodes
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewerForContentSummary
hadoop.hdfs.server.datanode.fsdataset.impl.TestFsDatasetCache
hadoop.hdfs.server.balancer.TestBalancerService
hadoop.hdfs.server.namenode.TestNNThroughputBenchmark
hadoop.hdfs.tools.TestECAdmin
hadoop.hdfs.TestLargeBlock
hadoop.hdfs.TestDFSStripedOutputStreamWithFailure
hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodeProtocol
hadoop.hdfs.server.namenode.TestINodeFile
hadoop.hdfs.server.datanode.TestBlockPoolManager
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyPersistPolicy
hadoop.hdfs.server.namenode.TestBlockPlacementPolicyRackFaultTolerant
hadoop.hdfs.TestReconstructStripedFile
hadoop.hdfs.server.namenode.ha.TestHAConfiguration
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewerForXAttr
hadoop.hdfs.TestFSInputChecker
hadoop.hdfs.TestDatanodeDeath
hadoop.hdfs.qjournal.TestMiniJournalCluster
hadoop.hdfs.server.blockmanagement.TestUnderReplicatedBlocks
hadoop.hdfs.server.namenode.TestFileLimit
hadoop.hdfs.server.datanode.TestNNHandlesBlockReportPerStorage
hadoop.hdfs.server.namenode.TestAddBlock
hadoop.hdfs.TestDFSInputStream
hadoop.hdfs.TestLease
hadoop.hdfs.TestTrashWithSecureEncryptionZones
hadoop.hdfs.TestReplaceDatanodeFailureReplication
hadoop.hdfs.server.namenode.TestDiskspaceQuotaUpdate
hadoop.hdfs.server.blockmanagement.TestCorruptionWithFailover
hadoop.hdfs.server.datanode.fsdataset.impl.TestPmemCacheRecovery
hadoop.hdfs.server.balancer.TestBalancerWithNodeGroup
hadoop.hdfs.server.blockmanagement.TestSequentialBlockId
hadoop.hdfs.server.datanode.TestDataStorage
hadoop.hdfs.server.namenode.TestEditLogRace
hadoop.fs.contract.hdfs.TestHDFSContractGetFileStatus
hadoop.hdfs.server.namenode.snapshot.TestAclWithSnapshot
hadoop.hdfs.TestDFSClientRetries
hadoop.hdfs.TestHAAuxiliaryPort
hadoop.hdfs.tools.TestDFSHAAdminMiniCluster
hadoop.hdfs.server.datanode.TestDataNodeTransferSocketSize
hadoop.hdfs.web.TestWebHDFSForHA
hadoop.hdfs.TestParallelUnixDomainRead
hadoop.hdfs.TestErasureCodingPolicies
hadoop.hdfs.shortcircuit.TestShortCircuitLocalRead
hadoop.hdfs.TestDFSStorageStateRecovery
hadoop.hdfs.server.blockmanagement.TestAvailableSpaceRackFaultTolerantBPP
hadoop.hdfs.server.blockmanagement.TestBlockManagerSafeMode
hadoop.hdfs.TestDatanodeRegistration
hadoop.hdfs.TestBlockStoragePolicy
hadoop.hdfs.TestReplaceDatanodeOnFailure
hadoop.hdfs.server.blockmanagement.TestBlockReportLease
hadoop.hdfs.server.blockmanagement.TestBlockInfoStriped
hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList
hadoop.hdfs.TestDFSRemove
hadoop.hdfs.TestCrcCorruption
hadoop.hdfs.TestFileCorruption
hadoop.hdfs.server.datanode.TestIncrementalBrVariations
hadoop.hdfs.server.namenode.TestNamenodeCapacityReport
hadoop.hdfs.server.datanode.checker.TestDatasetVolumeCheckerFailures
hadoop.hdfs.server.namenode.snapshot.TestSnapshotBlocksMap
hadoop.fs.viewfs.TestViewFileSystemLinkFallback
hadoop.hdfs.TestFileAppend3
hadoop.hdfs.qjournal.client.TestQuorumJournalManagerUnit
hadoop.hdfs.server.namenode.TestStorageRestore
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyPersistLockedMemory
hadoop.hdfs.TestReplication
hadoop.hdfs.server.blockmanagement.TestSlowDiskTracker
hadoop.hdfs.server.namenode.TestAuditLogger
hadoop.hdfs.TestDistributedFileSystemWithECFile
hadoop.hdfs.server.balancer.TestBalancerWithHANameNodes
hadoop.hdfs.server.datanode.TestDataNodeRollingUpgrade
hadoop.hdfs.server.blockmanagement.TestBlockPlacementPolicyDebugLoggingBuilder
hadoop.hdfs.TestFileStatusWithRandomECPolicy
hadoop.hdfs.server.datanode.TestBPOfferService
hadoop.hdfs.server.blockmanagement.TestSortLocatedStripedBlock
hadoop.hdfs.TestDFSClientExcludedNodes
hadoop.hdfs.qjournal.client.TestIPCLoggerChannel
hadoop.hdfs.qjournal.server.TestJournalNodeRespectsBindHostKeys
hadoop.hdfs.TestEncryptionZones
hadoop.cli.TestAclCLI
hadoop.hdfs.server.datanode.checker.TestDatasetVolumeCheckerTimeout
hadoop.hdfs.TestDFSClientSocketSize
hadoop.hdfs.server.namenode.ha.TestRetryCacheWithHA
hadoop.hdfs.server.namenode.snapshot.TestNestedSnapshots
hadoop.hdfs.server.datanode.TestCorruptMetadataFile
hadoop.hdfs.server.datanode.checker.TestDatasetVolumeChecker
hadoop.hdfs.web.TestWebHDFS
hadoop.fs.contract.hdfs.TestHDFSContractRename
hadoop.fs.TestHDFSFileContextMainOperations
hadoop.hdfs.TestDFSRollback
hadoop.hdfs.server.namenode.TestSaveNamespace
hadoop.fs.viewfs.TestViewFSOverloadSchemeWithMountTableConfigInHDFS
hadoop.cli.TestCacheAdminCLI
hadoop.hdfs.client.impl.TestBlockReaderIoProvider
hadoop.hdfs.client.impl.TestBlockReaderRemote
hadoop.hdfs.server.namenode.TestMalformedURLs
hadoop.hdfs.qjournal.client.TestEpochsAreUnique
hadoop.fs.viewfs.TestNNStartupWhenViewFSOverloadSchemeEnabled
hadoop.hdfs.TestDFSStripedInputStream
hadoop.hdfs.server.namenode.snapshot.TestSnapshotFileLength
hadoop.hdfs.web.TestWebHdfsUrl
hadoop.fs.viewfs.TestViewFsFileStatusHdfs
hadoop.tools.TestHdfsConfigFields
hadoop.hdfs.client.impl.TestBlockReaderLocalLegacy
hadoop.hdfs.server.namenode.TestNameNodeStatusMXBean
hadoop.hdfs.server.datanode.TestDeleteBlockPool
hadoop.hdfs.server.namenode.ha.TestFailoverWithBlockTokensEnabled
hadoop.hdfs.TestSnapshotCommands
hadoop.hdfs.qjournal.server.TestJournalNode
hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks
hadoop.hdfs.TestErasureCodingMultipleRacks
hadoop.cli.TestErasureCodingCLI
hadoop.hdfs.TestDataTransferKeepalive
hadoop.hdfs.server.blockmanagement.TestDatanodeManager
hadoop.fs.viewfs.TestViewFileSystemLinkMergeSlash
hadoop.hdfs.server.sps.TestExternalStoragePolicySatisfier
hadoop.hdfs.TestLocalDFS
hadoop.hdfs.TestClientProtocolForPipelineRecovery
hadoop.hdfs.TestWriteRead
hadoop.hdfs.TestFileAppend
hadoop.fs.TestSymlinkHdfsFileSystem
hadoop.hdfs.server.namenode.TestRefreshNamenodeReplicationConfig
hadoop.hdfs.TestDisableConnCache
hadoop.hdfs.TestMultipleNNPortQOP
hadoop.hdfs.server.diskbalancer.TestDiskBalancerRPC
hadoop.hdfs.web.TestWebHdfsWithAuthenticationFilter
hadoop.hdfs.server.namenode.TestNameNodeMetricsLogger
hadoop.hdfs.server.namenode.TestFSEditLogLoader
hadoop.hdfs.TestSecureEncryptionZoneWithKMS
hadoop.hdfs.server.namenode.TestCacheDirectives
hadoop.fs.permission.TestStickyBit
hadoop.hdfs.server.datanode.TestIncrementalBlockReports
hadoop.hdfs.server.datanode.TestDataNodeErasureCodingMetrics
hadoop.hdfs.server.datanode.TestDataNodeMXBean
hadoop.hdfs.server.namenode.ha.TestEditLogsDuringFailover
hadoop.hdfs.TestDatanodeLayoutUpgrade
hadoop.hdfs.server.namenode.ha.TestPendingCorruptDnMessages
hadoop.hdfs.TestRead
hadoop.hdfs.TestDecommissionWithBackoffMonitor
hadoop.hdfs.TestClientReportBadBlock
hadoop.hdfs.server.namenode.snapshot.TestSnapshotReplication
hadoop.hdfs.TestStripedFileAppend
hadoop.hdfs.TestViewDistributedFileSystem
hadoop.hdfs.server.namenode.snapshot.TestINodeFileUnderConstructionWithSnapshot
hadoop.fs.contract.hdfs.TestHDFSContractMultipartUploader
hadoop.fs.viewfs.TestViewFsWithXAttrs
hadoop.fs.TestWebHdfsFileContextMainOperations
hadoop.hdfs.web.TestHttpsFileSystem
hadoop.hdfs.TestRestartDFS
hadoop.hdfs.TestDistributedFileSystemWithECFileWithRandomECPolicy
hadoop.hdfs.server.namenode.TestNameNodeRpcServerMethods
hadoop.hdfs.server.datanode.TestProvidedReplicaImpl
hadoop.hdfs.server.datanode.web.webhdfs.TestDataNodeUGIProvider
hadoop.hdfs.server.blockmanagement.TestReplicationPolicy
hadoop.hdfs.server.datanode.TestDirectoryScanner
hadoop.hdfs.TestReconstructStripedFileWithRandomECPolicy
hadoop.hdfs.server.namenode.TestNameNodeRecovery
hadoop.hdfs.TestDFSStripedOutputStreamWithRandomECPolicy
hadoop.hdfs.TestExtendedAcls
hadoop.hdfs.TestErasureCodingPoliciesWithRandomECPolicy
hadoop.hdfs.TestFSOutputSummer
hadoop.hdfs.server.namenode.ha.TestUpdateBlockTailing
hadoop.hdfs.TestErasureCodingAddConfig
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDeletion
hadoop.hdfs.TestDFSUpgradeFromImage
hadoop.hdfs.server.namenode.ha.TestConsistentReadsObserver
hadoop.hdfs.web.TestWebHdfsWithMultipleNameNodes
hadoop.hdfs.server.namenode.TestGetImageServlet
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
hadoop.hdfs.server.blockmanagement.TestBlockStatsMXBean
hadoop.hdfs.TestSmallBlock
hadoop.hdfs.TestParallelShortCircuitLegacyRead
hadoop.hdfs.web.TestFSMainOperationsWebHdfs
hadoop.hdfs.server.namenode.TestDecommissioningStatus
hadoop.hdfs.TestDecommission
hadoop.hdfs.TestPipelines
hadoop.hdfs.server.namenode.TestNameNodeReconfigure
hadoop.hdfs.TestMiniDFSCluster
hadoop.hdfs.server.namenode.TestCacheDirectivesWithViewDFS
hadoop.hdfs.server.blockmanagement.TestRedundancyMonitor
hadoop.hdfs.TestAppendDifferentChecksum
hadoop.fs.contract.hdfs.TestHDFSContractRootDirectory
hadoop.fs.contract.hdfs.TestHDFSContractSeek
hadoop.hdfs.server.datanode.TestTransferRbw
hadoop.hdfs.tools.TestViewFSStoragePolicyCommands
hadoop.hdfs.TestDFSStripedOutputStream
hadoop.hdfs.tools.TestDFSAdmin
hadoop.hdfs.server.mover.TestStorageMover
hadoop.hdfs.server.namenode.ha.TestHarFileSystemWithHA
hadoop.security.TestPermissionSymlinks
hadoop.hdfs.server.namenode.snapshot.TestUpdatePipelineWithSnapshots
hadoop.hdfs.server.namenode.ha.TestStandbyInProgressTail
hadoop.hdfs.TestDFSUpgrade
hadoop.hdfs.server.datanode.TestDataNodeInitStorage
hadoop.hdfs.server.blockmanagement.TestReconstructStripedBlocksWithRackAwareness
hadoop.fs.TestUrlStreamHandler
hadoop.hdfs.server.namenode.TestNameNodeXAttr
hadoop.hdfs.web.TestWebHdfsWithRestCsrfPreventionFilter
hadoop.hdfs.server.datanode.TestSimulatedFSDataset
hadoop.hdfs.server.blockmanagement.TestBlockReportRateLimiting
hadoop.hdfs.server.datanode.TestStartSecureDataNode
hadoop.hdfs.server.namenode.TestValidateConfigurationSettings
hadoop.hdfs.server.namenode.web.resources.TestWebHdfsCreatePermissions
hadoop.fs.shell.TestHdfsTextCommand
hadoop.hdfs.server.mover.TestMover
hadoop.hdfs.server.namenode.TestXAttrConfigFlag
hadoop.hdfs.server.namenode.TestNameEditsConfigs
hadoop.security.TestPermission
hadoop.hdfs.server.datanode.fsdataset.impl.TestProvidedImpl
hadoop.hdfs.TestGetFileChecksum
hadoop.hdfs.server.namenode.snapshot.TestCheckpointsWithSnapshots
hadoop.hdfs.server.blockmanagement.TestReplicationPolicyExcludeSlowNodes
hadoop.hdfs.server.datanode.fsdataset.impl.TestFsDatasetImpl
hadoop.hdfs.server.namenode.TestFileJournalManager
hadoop.hdfs.server.blockmanagement.TestBlockManager
hadoop.hdfs.server.namenode.snapshot.TestListSnapshot
hadoop.hdfs.server.aliasmap.TestSecureAliasMap
hadoop.hdfs.server.namenode.TestDeleteRace
hadoop.hdfs.web.TestWebHDFSXAttr
hadoop.hdfs.client.impl.TestBlockReaderLocal
hadoop.hdfs.server.namenode.sps.TestBlockStorageMovementAttemptedItems
hadoop.hdfs.TestHDFSFileSystemContract
hadoop.hdfs.TestParallelShortCircuitReadNoChecksum
hadoop.hdfs.TestFileAppend4
hadoop.hdfs.TestBlockTokenWrappingQOP
hadoop.hdfs.server.namenode.snapshot.TestRandomOpsWithSnapshots
hadoop.hdfs.server.namenode.TestSecondaryNameNodeUpgrade
hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics
hadoop.hdfs.TestReadStripedFileWithMissingBlocks
hadoop.hdfs.server.blockmanagement.TestErasureCodingCorruption
hadoop.fs.contract.hdfs.TestHDFSContractAppend
hadoop.hdfs.TestReadStripedFileWithDecoding
hadoop.hdfs.server.namenode.ha.TestNNHealthCheck
hadoop.hdfs.server.namenode.TestReencryption
hadoop.hdfs.TestFileStatusWithDefaultECPolicy
hadoop.hdfs.TestBalancerBandwidth
hadoop.hdfs.server.namenode.snapshot.TestSnapRootDescendantDiff
hadoop.hdfs.TestErasureCodingExerciseAPIs
hadoop.hdfs.tools.TestViewFileSystemOverloadSchemeWithDFSAdmin
hadoop.hdfs.TestWriteReadStripedFile
hadoop.hdfs.server.namenode.TestFileContextAcl
hadoop.hdfs.server.namenode.TestSnapshotPathINodes
hadoop.hdfs.TestQuota
hadoop.hdfs.server.namenode.TestClusterId
hadoop.hdfs.server.namenode.TestBackupNode
hadoop.hdfs.server.blockmanagement.TestAvailableSpaceBPPBalanceLocal
hadoop.hdfs.server.namenode.TestMetadataVersionOutput
hadoop.hdfs.server.namenode.TestDeadDatanode
hadoop.hdfs.TestReservedRawPaths
hadoop.hdfs.server.namenode.ha.TestDNFencing
hadoop.hdfs.server.namenode.TestGenericJournalConf
hadoop.hdfs.TestBlocksScheduledCounter
hadoop.hdfs.server.namenode.TestNameNodeResourceChecker
hadoop.hdfs.server.namenode.snapshot.TestSnapshot
hadoop.hdfs.server.namenode.TestTransferFsImage
hadoop.hdfs.TestRenameWhileOpen
hadoop.hdfs.server.namenode.metrics.TestNNMetricFilesInGetListingOps
hadoop.hdfs.tools.TestDebugAdmin
hadoop.fs.viewfs.TestViewFsWithAcls
hadoop.hdfs.server.datanode.TestBlockRecovery
hadoop.hdfs.server.namenode.TestFSNamesystemMBean
hadoop.hdfs.TestParallelShortCircuitReadUnCached
hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints
hadoop.hdfs.TestHDFSServerPorts
hadoop.hdfs.server.datanode.TestReadOnlySharedStorage
hadoop.hdfs.TestFileCreationClient
hadoop.hdfs.server.namenode.snapshot.TestRenameWithSnapshots
hadoop.hdfs.server.namenode.TestProtectedDirectories
hadoop.hdfs.TestLeaseRecovery2
hadoop.hdfs.server.datanode.TestDataNodeVolumeFailure
hadoop.hdfs.TestQuotaAllowOwner
hadoop.hdfs.TestListFilesInDFS
hadoop.hdfs.TestDFSPermission
hadoop.TestRefreshCallQueue
hadoop.hdfs.TestDatanodeReport
hadoop.hdfs.server.namenode.TestPersistentStoragePolicySatisfier
hadoop.hdfs.server.blockmanagement.TestOverReplicatedBlocks
hadoop.hdfs.TestSetrepIncreasing
hadoop.hdfs.server.namenode.ha.TestObserverNode
hadoop.hdfs.web.TestWebHdfsTimeouts
hadoop.hdfs.server.namenode.TestFileTruncate
hadoop.hdfs.server.balancer.TestBalancerWithSaslDataTransfer
hadoop.hdfs.server.datanode.TestFsDatasetCacheRevocation
hadoop.hdfs.server.namenode.TestListOpenFiles
hadoop.hdfs.server.blockmanagement.TestHostFileManager
hadoop.hdfs.server.namenode.ha.TestMultiObserverNode
hadoop.fs.contract.hdfs.TestHDFSContractUnbuffer
hadoop.hdfs.server.namenode.TestFSNamesystem
hadoop.hdfs.server.datanode.fsdataset.impl.TestReplicaCachingGetSpaceUsed
hadoop.hdfs.TestBlockMissingException
hadoop.hdfs.tools.offlineEditsViewer.TestOfflineEditsViewer
hadoop.hdfs.tools.TestDFSAdminWithHA
hadoop.hdfs.TestDFSMkdirs
hadoop.hdfs.server.namenode.TestFSPermissionChecker
hadoop.hdfs.server.namenode.ha.TestFailureOfSharedDir
hadoop.hdfs.server.namenode.TestEnabledECPolicies
hadoop.hdfs.server.common.TestJspHelper
hadoop.hdfs.server.namenode.TestCommitBlockSynchronization
hadoop.hdfs.server.datanode.TestDataNodeMultipleRegistrations
hadoop.hdfs.server.namenode.TestSecurityTokenEditLog
hadoop.hdfs.TestHdfsAdmin
hadoop.hdfs.TestReadStripedFileWithDecodingCorruptData
hadoop.hdfs.server.namenode.ha.TestPipelinesFailover
hadoop.hdfs.server.namenode.snapshot.TestSnapshottableDirListing
hadoop.hdfs.server.datanode.TestNNHandlesCombinedBlockReport
hadoop.hdfs.server.datanode.TestDataNodeECN
hadoop.hdfs.server.namenode.TestAclConfigFlag
hadoop.hdfs.TestLeaseRecovery
hadoop.hdfs.server.blockmanagement.TestReplicationPolicyConsiderLoad
hadoop.hdfs.TestHDFSTrash
hadoop.hdfs.web.TestWebHdfsFileSystemContract
hadoop.hdfs.server.datanode.TestDataNodeVolumeMetrics
hadoop.hdfs.server.namenode.TestHDFSConcat
hadoop.hdfs.security.TestDelegationToken
hadoop.fs.contract.hdfs.TestHDFSContractDelete
hadoop.hdfs.TestDFSInotifyEventInputStreamKerberized
hadoop.hdfs.tools.TestGetGroups
hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor
hadoop.hdfs.TestGetBlocks
hadoop.hdfs.server.datanode.TestDataXceiverBackwardsCompat
hadoop.hdfs.server.namenode.ha.TestEditLogTailer
hadoop.hdfs.server.datanode.fsdataset.impl.TestDatanodeRestart
hadoop.fs.viewfs.TestViewFsDefaultValue
hadoop.hdfs.TestModTime
hadoop.hdfs.server.namenode.TestRedudantBlocks
hadoop.hdfs.server.namenode.TestParallelImageWrite
hadoop.hdfs.server.blockmanagement.TestAvailableSpaceBlockPlacementPolicy
hadoop.hdfs.server.namenode.TestNameNodeHttpServerXFrame
hadoop.hdfs.server.namenode.TestReencryptionHandler
hadoop.hdfs.server.namenode.TestCreateEditsLog
hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion
hadoop.security.TestRefreshUserMappings
hadoop.hdfs.server.namenode.TestFsLimits
hadoop.hdfs.server.namenode.TestProcessCorruptBlocks
hadoop.hdfs.server.namenode.ha.TestGetGroupsWithHA
hadoop.hdfs.server.namenode.TestEditLog
hadoop.hdfs.TestDFSOutputStream
hadoop.hdfs.server.namenode.ha.TestXAttrsWithHA
hadoop.fs.loadGenerator.TestLoadGenerator
hadoop.hdfs.tools.TestStoragePolicySatisfyAdminCommands
hadoop.fs.contract.hdfs.TestHDFSContractCreate
hadoop.hdfs.TestDFSAddressConfig
hadoop.hdfs.server.datanode.TestDataNodeLifeline
hadoop.hdfs.server.namenode.TestNameNodeRpcServer
hadoop.hdfs.server.namenode.snapshot.TestDisallowModifyROSnapshot
hadoop.hdfs.server.blockmanagement.TestProvidedStorageMap
hadoop.hdfs.TestEncryptionZonesWithHA
hadoop.hdfs.server.namenode.ha.TestHAAppend
hadoop.hdfs.TestDFSFinalize
hadoop.hdfs.shortcircuit.TestShortCircuitCache
hadoop.fs.TestUnbuffer
hadoop.hdfs.server.namenode.snapshot.TestSnapshotManager
hadoop.fs.viewfs.TestViewFsAtHdfsRoot
hadoop.hdfs.client.impl.TestBlockReaderLocalMetrics
hadoop.hdfs.server.namenode.TestMetaSave
hadoop.fs.contract.hdfs.TestHDFSContractPathHandle
hadoop.hdfs.TestReconstructStripedFileWithValidator
hadoop.hdfs.TestMultiThreadedHflush
hadoop.hdfs.TestReadStripedFileWithDecodingDeletedData
hadoop.hdfs.protocol.datatransfer.sasl.TestSaslDataTransfer
hadoop.hdfs.crypto.TestHdfsCryptoStreams
hadoop.hdfs.server.datanode.TestBatchIbr
hadoop.hdfs.TestConnCache
hadoop.hdfs.server.datanode.TestDatanodeProtocolRetryPolicy
hadoop.hdfs.TestTrashWithEncryptionZones
hadoop.hdfs.server.datanode.TestDataXceiverLazyPersistHint
hadoop.hdfs.server.datanode.TestHSync
hadoop.hdfs.server.namenode.TestLargeDirectoryDelete
hadoop.hdfs.tools.TestDFSHAAdmin
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyPersistReplicaPlacement
hadoop.hdfs.server.namenode.TestFSNamesystemLockReport
hadoop.hdfs.server.namenode.TestFSImageWithXAttr
hadoop.fs.TestFcHdfsCreateMkdir
hadoop.hdfs.TestDFSShell
hadoop.fs.viewfs.TestViewFileSystemWithAcls
hadoop.fs.contract.hdfs.TestHDFSContractOpen
hadoop.hdfs.tools.TestWebHDFSStoragePolicyCommands
hadoop.hdfs.TestHFlush
hadoop.hdfs.server.namenode.TestFSDirectory
hadoop.hdfs.tools.TestGetConf
hadoop.hdfs.server.namenode.TestINodeAttributeProvider
hadoop.hdfs.server.namenode.TestGetContentSummaryWithPermission
hadoop.hdfs.server.namenode.TestQuotaByStorageType
hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped
hadoop.hdfs.server.diskbalancer.TestDiskBalancer
hadoop.hdfs.server.namenode.snapshot.TestSnapshotStatsMXBean
hadoop.hdfs.server.datanode.fsdataset.impl.TestCacheByPmemMappableBlockLoader
hadoop.hdfs.TestWriteConfigurationToDFS
hadoop.hdfs.server.namenode.TestClientNameNodeAddress
hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.TestWriteBlockGetsBlockLengthHint
hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFS
hadoop.hdfs.server.datanode.checker.TestStorageLocationChecker
hadoop.hdfs.server.namenode.snapshot.TestFSImageWithOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.ha.TestQuotasWithHA
hadoop.hdfs.TestFileStatus
hadoop.cli.TestCryptoAdminCLI
hadoop.hdfs.TestSeekBug
hadoop.hdfs.server.namenode.TestNestedEncryptionZones
hadoop.hdfs.server.datanode.TestBlockCountersInPendingIBR
hadoop.fs.viewfs.TestViewFileSystemLinkRegex
hadoop.hdfs.server.namenode.TestAllowFormat
hadoop.hdfs.TestDFSStripedOutputStreamUpdatePipeline
hadoop.hdfs.server.namenode.TestStoragePolicySatisfierWithHA
hadoop.hdfs.server.namenode.TestSecondaryWebUi
hadoop.hdfs.server.datanode.TestDataNodeTcpNoDelay
hadoop.hdfs.server.datanode.web.TestDatanodeHttpXFrame
hadoop.cli.TestXAttrCLI
hadoop.hdfs.TestSafeModeWithStripedFileWithRandomECPolicy
hadoop.hdfs.server.namenode.snapshot.TestSnapshotListing
hadoop.hdfs.server.datanode.TestBlockRecovery2
hadoop.hdfs.TestReadWhileWriting
hadoop.hdfs.server.namenode.TestRefreshBlockPlacementPolicy
hadoop.hdfs.server.namenode.ha.TestBootstrapStandby
hadoop.hdfs.server.datanode.TestCachingStrategy
hadoop.hdfs.server.namenode.TestAddStripedBlocks
hadoop.hdfs.TestReadStripedFileWithDNFailure
hadoop.fs.TestResolveHdfsSymlink
hadoop.hdfs.server.namenode.TestCheckPointForSecurityTokens
hadoop.hdfs.TestMissingBlocksAlert
hadoop.hdfs.server.datanode.TestBlockScanner
hadoop.hdfs.server.datanode.fsdataset.impl.TestWriteToReplica
hadoop.hdfs.server.balancer.TestBalancerWithEncryptedTransfer
hadoop.hdfs.client.impl.TestClientBlockVerification
hadoop.hdfs.server.blockmanagement.TestReplicationPolicyWithUpgradeDomain
hadoop.hdfs.web.TestWebHDFSAcl
hadoop.hdfs.TestBatchedListDirectories
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/1/artifact/out/Dockerfile
GITHUB PR #3417
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux 3cba315f2347 4.15.0-151-generic #157-Ubuntu SMP Fri Jul 9 23:07:57 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision fgl / 6b98ab926892e14eb759a453f4aa6145b1479266
Default Java Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/1/testReport/
Max. process+thread count 753 (vs. ulimit of 5500)
modules C: hadoop-hdfs-project/hadoop-hdfs U: hadoop-hdfs-project/hadoop-hdfs
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/1/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 1m 8s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 1 new or modified test files.
_ fgl Compile Tests _
+1 💚 mvninstall 38m 34s fgl passed
+1 💚 compile 1m 25s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 compile 1m 16s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 checkstyle 1m 0s fgl passed
+1 💚 mvnsite 1m 24s fgl passed
+1 💚 javadoc 0m 56s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 1m 23s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 spotbugs 3m 17s fgl passed
+1 💚 shadedclient 19m 6s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 17s the patch passed
+1 💚 compile 1m 18s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javac 1m 18s the patch passed
+1 💚 compile 1m 8s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 javac 1m 8s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 54s /results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs-project/hadoop-hdfs: The patch generated 10 new + 285 unchanged - 0 fixed = 295 total (was 285)
+1 💚 mvnsite 1m 17s the patch passed
+1 💚 xml 0m 1s The patch has no ill-formed XML file.
+1 💚 javadoc 0m 49s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 1m 18s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ spotbugs 3m 27s /new-spotbugs-hadoop-hdfs-project_hadoop-hdfs.html hadoop-hdfs-project/hadoop-hdfs generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)
+1 💚 shadedclient 19m 22s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 433m 45s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 37s The patch does not generate ASF License warnings.
532m 23s
Reason Tests
SpotBugs module:hadoop-hdfs-project/hadoop-hdfs
Write to static field org.apache.hadoop.hdfs.server.namenode.INodeMap.namespaceKeyDepth from instance method new org.apache.hadoop.hdfs.server.namenode.INodeMap(INodeDirectory, FSNamesystem, Configuration) At INodeMap.java:from instance method new org.apache.hadoop.hdfs.server.namenode.INodeMap(INodeDirectory, FSNamesystem, Configuration) At INodeMap.java:[line 206]
Failed junit tests hadoop.hdfs.server.namenode.snapshot.TestCheckpointsWithSnapshots
hadoop.hdfs.server.namenode.TestStorageRestore
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDeletion
hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics
hadoop.hdfs.server.namenode.ha.TestPipelinesFailover
hadoop.hdfs.TestDFSUpgradeFromImage
hadoop.hdfs.server.namenode.TestSecondaryNameNodeUpgrade
hadoop.hdfs.TestAppendSnapshotTruncate
hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeHdfsFileSystemContract
hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.TestRollingUpgradeDowngrade
hadoop.hdfs.server.namenode.snapshot.TestFSImageWithOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.TestFSNamesystem
hadoop.hdfs.server.namenode.snapshot.TestAclWithSnapshot
hadoop.hdfs.server.namenode.TestFSImage
hadoop.hdfs.server.namenode.snapshot.TestSnapRootDescendantDiff
hadoop.hdfs.server.namenode.snapshot.TestRenameWithSnapshots
hadoop.hdfs.server.namenode.TestCommitBlockSynchronization
hadoop.hdfs.TestDistributedFileSystem
hadoop.hdfs.server.namenode.ha.TestEditLogTailer
hadoop.hdfs.server.namenode.TestStartup
hadoop.hdfs.server.namenode.ha.TestObserverNode
hadoop.hdfs.server.namenode.TestNameEditsConfigs
hadoop.hdfs.server.namenode.snapshot.TestXAttrWithSnapshot
hadoop.hdfs.TestRollingUpgradeRollback
hadoop.hdfs.server.datanode.TestBlockScanner
hadoop.hdfs.server.namenode.TestFSNamesystemLock
hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor
hadoop.hdfs.server.namenode.TestFSNamesystemLockReport
hadoop.hdfs.server.namenode.snapshot.TestOpenFilesWithSnapshot
hadoop.hdfs.server.namenode.TestFSImageWithSnapshot
hadoop.hdfs.server.namenode.TestNameNodeRecovery
hadoop.hdfs.TestErasureCodingPolicyWithSnapshot
hadoop.hdfs.server.namenode.ha.TestFailureToReadEdits
hadoop.hdfs.server.namenode.TestAuditLogs
hadoop.hdfs.TestViewDistributedFileSystem
hadoop.hdfs.TestErasureCodingPolicyWithSnapshotWithRandomECPolicy
hadoop.hdfs.server.namenode.TestDecommissioningStatus
hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.TestCacheDirectives
hadoop.hdfs.server.datanode.fsdataset.impl.TestFsDatasetImpl
hadoop.hdfs.server.namenode.snapshot.TestSnapshot
hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints
hadoop.hdfs.server.namenode.snapshot.TestSnapshotBlocksMap
hadoop.hdfs.server.namenode.TestDeleteRace
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
hadoop.hdfs.server.namenode.TestFileTruncate
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDiffReport
hadoop.hdfs.server.namenode.TestCacheDirectivesWithViewDFS
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/2/artifact/out/Dockerfile
GITHUB PR #3417
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux 1a3343afc089 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision fgl / 0501b8729aba4cd424894ca83644c7156a2bd143
Default Java Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/2/testReport/
Max. process+thread count 1925 (vs. ulimit of 5500)
modules C: hadoop-hdfs-project/hadoop-hdfs U: hadoop-hdfs-project/hadoop-hdfs
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/2/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 45s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 1 new or modified test files.
_ fgl Compile Tests _
+1 💚 mvninstall 31m 33s fgl passed
+1 💚 compile 1m 23s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 compile 1m 17s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 checkstyle 1m 2s fgl passed
+1 💚 mvnsite 1m 26s fgl passed
+1 💚 javadoc 0m 57s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 1m 28s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 spotbugs 3m 11s fgl passed
+1 💚 shadedclient 16m 29s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 1m 10s the patch passed
+1 💚 compile 1m 13s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javac 1m 13s the patch passed
+1 💚 compile 1m 11s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 javac 1m 11s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 55s /results-checkstyle-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs-project/hadoop-hdfs: The patch generated 17 new + 285 unchanged - 0 fixed = 302 total (was 285)
+1 💚 mvnsite 1m 15s the patch passed
+1 💚 xml 0m 1s The patch has no ill-formed XML file.
+1 💚 javadoc 0m 46s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 1m 20s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 spotbugs 3m 11s the patch passed
+1 💚 shadedclient 16m 17s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 286m 27s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 0m 52s The patch does not generate ASF License warnings.
372m 50s
Reason Tests
Failed junit tests hadoop.hdfs.server.namenode.snapshot.TestCheckpointsWithSnapshots
hadoop.hdfs.server.namenode.snapshot.TestFSImageWithOrderedSnapshotDeletion
hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeHdfsFileSystemContract
hadoop.hdfs.server.namenode.snapshot.TestRandomOpsWithSnapshots
hadoop.hdfs.TestFileChecksum
hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints
hadoop.hdfs.server.namenode.TestFSNamesystemLock
hadoop.hdfs.TestDFSUpgrade
hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.server.namenode.TestNameNodeRecovery
hadoop.hdfs.server.namenode.TestStartup
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
hadoop.hdfs.server.namenode.snapshot.TestOpenFilesWithSnapshot
hadoop.hdfs.server.namenode.TestFSImage
hadoop.hdfs.TestDFSUpgradeFromImage
hadoop.hdfs.TestViewDistributedFileSystem
hadoop.hdfs.server.namenode.TestStorageRestore
hadoop.hdfs.TestRollingUpgradeDowngrade
hadoop.hdfs.server.namenode.TestFSNamesystem
hadoop.hdfs.server.namenode.snapshot.TestSnapRootDescendantDiff
hadoop.hdfs.server.namenode.TestFSNamesystemLockReport
hadoop.hdfs.server.namenode.ha.TestFailureToReadEdits
hadoop.hdfs.server.namenode.snapshot.TestSnapshotBlocksMap
hadoop.hdfs.server.namenode.TestDiskspaceQuotaUpdate
hadoop.hdfs.server.namenode.TestNameEditsConfigs
hadoop.hdfs.server.namenode.TestDeleteRace
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDiffReport
hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDeletion
hadoop.hdfs.server.namenode.TestCacheDirectivesWithViewDFS
hadoop.hdfs.server.namenode.snapshot.TestAclWithSnapshot
hadoop.hdfs.TestRollingUpgradeRollback
hadoop.hdfs.TestErasureCodingPolicyWithSnapshot
hadoop.hdfs.TestDistributedFileSystem
hadoop.hdfs.server.namenode.TestFSImageWithSnapshot
hadoop.hdfs.server.namenode.TestSecondaryNameNodeUpgrade
hadoop.hdfs.server.namenode.snapshot.TestSnapshot
hadoop.hdfs.server.namenode.TestFileTruncate
hadoop.hdfs.server.namenode.TestCacheDirectives
hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics
hadoop.hdfs.server.namenode.snapshot.TestXAttrWithSnapshot
hadoop.hdfs.server.namenode.ha.TestSeveralNameNodes
hadoop.hdfs.TestErasureCodingPolicyWithSnapshotWithRandomECPolicy
hadoop.hdfs.server.namenode.TestCommitBlockSynchronization
hadoop.hdfs.TestViewDistributedFileSystemContract
hadoop.hdfs.TestSnapshotCommands
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/3/artifact/out/Dockerfile
GITHUB PR #3417
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux 9e5b7cbdac86 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision fgl / ba8541f48095a20e280d0fbfa4dd06abdb4b9ea2
Default Java Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/3/testReport/
Max. process+thread count 2843 (vs. ulimit of 5500)
modules C: hadoop-hdfs-project/hadoop-hdfs U: hadoop-hdfs-project/hadoop-hdfs
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/3/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

Copy link
Contributor

@xinglin xinglin left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking good overall. Thanks for working on this!

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is definitely one way to do it but is there a way we can make numSpaceKeyDepth/numRangesStatic a non-static variable?

numspaceKeyDepth -> numSpaceKeyDepth
numRangesStatic -> numRanges

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @xinglin for the comment.
I will update it later.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, if we want to support namespaceKeyDepth other than 2, we probably need to modify the range Keys we insert when we create new partitions. Instead of inserting range key such as [0, 16385], [1, 16385], [2, 16385], I think we might need to insert range keys as [0,0, 16385], [1,0,16385], [2,0,16385] ... for depth of 3 and [0,0,0,16385], [1,0,0,16385], [2,0,0,16385]... for depth of 4.

    for (int p = 0; p < numRangesStatic; p++) {
      INodeDirectory key = new INodeDirectory(INodeId.ROOT_INODE_ID,
          "range key".getBytes(StandardCharsets.UTF_8), perm, 0);
      key.setParent(new INodeDirectory((long)p, null, perm, 0));

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @xinglin for the comment.
I agree with you and I will update it later.

@jianghuazhu jianghuazhu force-pushed the HDFS-16220 branch 4 times, most recently from 8dae87f to 76553d1 Compare September 14, 2021 15:01
@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 46s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 1 new or modified test files.
_ fgl Compile Tests _
+0 🆗 mvndep 12m 38s Maven dependency ordering for branch
+1 💚 mvninstall 21m 51s fgl passed
+1 💚 compile 23m 27s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 compile 18m 35s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 checkstyle 3m 39s fgl passed
+1 💚 mvnsite 3m 15s fgl passed
+1 💚 javadoc 2m 19s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 23s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ spotbugs 2m 23s /branch-spotbugs-hadoop-common-project_hadoop-common-warnings.html hadoop-common-project/hadoop-common in fgl has 2 extant spotbugs warnings.
+1 💚 shadedclient 16m 56s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 27s Maven dependency ordering for patch
+1 💚 mvninstall 2m 9s the patch passed
+1 💚 compile 20m 56s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
-1 ❌ javac 20m 56s /results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1915 unchanged - 1 fixed = 1916 total (was 1916)
+1 💚 compile 19m 27s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ javac 19m 27s /results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu120.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu120.04-b10 generated 1 new + 1791 unchanged - 1 fixed = 1792 total (was 1792)
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 3m 30s /results-checkstyle-root.txt root: The patch generated 42 new + 291 unchanged - 0 fixed = 333 total (was 291)
+1 💚 mvnsite 3m 8s the patch passed
+1 💚 xml 0m 2s The patch has no ill-formed XML file.
+1 💚 javadoc 2m 11s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 9s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 spotbugs 6m 7s the patch passed
+1 💚 shadedclient 16m 58s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 17m 11s hadoop-common in the patch passed.
-1 ❌ unit 280m 46s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 1m 14s The patch does not generate ASF License warnings.
489m 7s
Reason Tests
Failed junit tests hadoop.hdfs.server.namenode.TestAuditLogs
hadoop.hdfs.server.namenode.snapshot.TestCheckpointsWithSnapshots
hadoop.hdfs.server.namenode.snapshot.TestFSImageWithOrderedSnapshotDeletion
hadoop.hdfs.TestStateAlignmentContextWithHA
hadoop.hdfs.server.namenode.snapshot.TestRandomOpsWithSnapshots
hadoop.hdfs.TestFileChecksum
hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints
hadoop.hdfs.server.namenode.TestFSNamesystemLock
hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.server.namenode.TestNameNodeRecovery
hadoop.hdfs.server.namenode.TestStartup
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
hadoop.tools.TestHdfsConfigFields
hadoop.hdfs.server.namenode.snapshot.TestOpenFilesWithSnapshot
hadoop.hdfs.server.namenode.TestFSImage
hadoop.hdfs.TestDFSUpgradeFromImage
hadoop.hdfs.TestViewDistributedFileSystem
hadoop.hdfs.server.namenode.TestStorageRestore
hadoop.hdfs.TestRollingUpgradeDowngrade
hadoop.hdfs.server.blockmanagement.TestReplicationPolicy
hadoop.hdfs.server.namenode.TestFSNamesystem
hadoop.hdfs.server.namenode.snapshot.TestSnapRootDescendantDiff
hadoop.hdfs.server.namenode.TestFSNamesystemLockReport
hadoop.hdfs.TestParallelRead
hadoop.hdfs.server.namenode.ha.TestFailureToReadEdits
hadoop.hdfs.server.namenode.snapshot.TestSnapshotBlocksMap
hadoop.hdfs.server.balancer.TestBalancerWithHANameNodes
hadoop.hdfs.server.namenode.TestNameEditsConfigs
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDiffReport
hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDeletion
hadoop.hdfs.server.namenode.TestCacheDirectivesWithViewDFS
hadoop.hdfs.server.namenode.snapshot.TestAclWithSnapshot
hadoop.hdfs.TestRollingUpgradeRollback
hadoop.hdfs.TestErasureCodingPolicyWithSnapshot
hadoop.hdfs.TestDistributedFileSystem
hadoop.hdfs.server.namenode.TestFSImageWithSnapshot
hadoop.hdfs.server.namenode.TestSecondaryNameNodeUpgrade
hadoop.hdfs.server.namenode.snapshot.TestSnapshot
hadoop.hdfs.server.namenode.TestFileTruncate
hadoop.hdfs.server.namenode.ha.TestObserverNode
hadoop.hdfs.server.namenode.TestCacheDirectives
hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics
hadoop.hdfs.server.namenode.snapshot.TestXAttrWithSnapshot
hadoop.hdfs.TestErasureCodingPolicyWithSnapshotWithRandomECPolicy
hadoop.hdfs.server.namenode.TestCommitBlockSynchronization
hadoop.hdfs.TestViewDistributedFileSystemContract
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/5/artifact/out/Dockerfile
GITHUB PR #3417
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux 859eaf9f9ef9 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision fgl / bd89524240f13a094684b36c27dfa13f7b697194
Default Java Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/5/testReport/
Max. process+thread count 3587 (vs. ulimit of 5500)
modules C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/5/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 57s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 1 new or modified test files.
_ fgl Compile Tests _
+0 🆗 mvndep 12m 27s Maven dependency ordering for branch
+1 💚 mvninstall 22m 53s fgl passed
+1 💚 compile 24m 51s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 compile 21m 6s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 checkstyle 4m 40s fgl passed
+1 💚 mvnsite 3m 28s fgl passed
+1 💚 javadoc 2m 23s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 22s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ spotbugs 2m 31s /branch-spotbugs-hadoop-common-project_hadoop-common-warnings.html hadoop-common-project/hadoop-common in fgl has 2 extant spotbugs warnings.
+1 💚 shadedclient 20m 51s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 25s Maven dependency ordering for patch
+1 💚 mvninstall 2m 32s the patch passed
+1 💚 compile 24m 49s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
-1 ❌ javac 24m 49s /results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1915 unchanged - 1 fixed = 1916 total (was 1916)
+1 💚 compile 20m 43s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ javac 20m 43s /results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu120.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu120.04-b10 generated 1 new + 1791 unchanged - 1 fixed = 1792 total (was 1792)
+1 💚 blanks 0m 1s The patch has no blanks issues.
-0 ⚠️ checkstyle 4m 23s /results-checkstyle-root.txt root: The patch generated 42 new + 291 unchanged - 0 fixed = 333 total (was 291)
+1 💚 mvnsite 3m 32s the patch passed
+1 💚 xml 0m 2s The patch has no ill-formed XML file.
+1 💚 javadoc 2m 24s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 13s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 spotbugs 6m 49s the patch passed
+1 💚 shadedclient 21m 16s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 17m 45s hadoop-common in the patch passed.
-1 ❌ unit 456m 51s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 1m 54s The patch does not generate ASF License warnings.
688m 1s
Reason Tests
Failed junit tests hadoop.hdfs.server.namenode.snapshot.TestCheckpointsWithSnapshots
hadoop.hdfs.server.namenode.TestStorageRestore
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDeletion
hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics
hadoop.hdfs.TestDFSUpgradeFromImage
hadoop.hdfs.server.namenode.TestSecondaryNameNodeUpgrade
hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeHdfsFileSystemContract
hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.TestRollingUpgradeDowngrade
hadoop.hdfs.server.namenode.snapshot.TestFSImageWithOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.TestFSNamesystem
hadoop.hdfs.server.namenode.snapshot.TestAclWithSnapshot
hadoop.hdfs.server.namenode.TestFSImage
hadoop.hdfs.TestDFSUpgrade
hadoop.hdfs.server.namenode.snapshot.TestSnapRootDescendantDiff
hadoop.hdfs.server.namenode.snapshot.TestRenameWithSnapshots
hadoop.hdfs.server.namenode.TestCommitBlockSynchronization
hadoop.hdfs.TestDistributedFileSystem
hadoop.hdfs.server.namenode.ha.TestEditLogTailer
hadoop.hdfs.server.namenode.TestStartup
hadoop.hdfs.server.namenode.TestNameEditsConfigs
hadoop.hdfs.server.namenode.snapshot.TestXAttrWithSnapshot
hadoop.hdfs.TestRollingUpgradeRollback
hadoop.hdfs.server.namenode.TestFSNamesystemLock
hadoop.tools.TestHdfsConfigFields
hadoop.hdfs.TestViewDistributedFileSystemWithMountLinks
hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor
hadoop.hdfs.server.namenode.TestFSNamesystemLockReport
hadoop.hdfs.TestSnapshotCommands
hadoop.hdfs.server.namenode.snapshot.TestOpenFilesWithSnapshot
hadoop.hdfs.server.namenode.TestFSImageWithSnapshot
hadoop.hdfs.TestErasureCodingPolicyWithSnapshot
hadoop.fs.contract.hdfs.TestHDFSContractMkdir
hadoop.hdfs.server.namenode.ha.TestFailureToReadEdits
hadoop.hdfs.TestViewDistributedFileSystem
hadoop.hdfs.TestErasureCodingPolicyWithSnapshotWithRandomECPolicy
hadoop.hdfs.TestDFSRemove
hadoop.hdfs.server.namenode.TestDecommissioningStatus
hadoop.hdfs.TestHDFSFileSystemContract
hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.TestCacheDirectives
hadoop.hdfs.server.namenode.TestPersistentStoragePolicySatisfier
hadoop.hdfs.server.namenode.snapshot.TestSnapshot
hadoop.hdfs.server.namenode.snapshot.TestSnapshotBlocksMap
hadoop.hdfs.server.namenode.TestDeleteRace
hadoop.hdfs.TestViewDistributedFileSystemContract
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
hadoop.fs.viewfs.TestViewFSOverloadSchemeWithMountTableConfigInHDFS
hadoop.hdfs.server.namenode.TestFileTruncate
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDiffReport
hadoop.hdfs.server.namenode.TestCacheDirectivesWithViewDFS
hadoop.hdfs.server.namenode.TestNamenodeStorageDirectives
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/4/artifact/out/Dockerfile
GITHUB PR #3417
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux 79b83699a3bc 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision fgl / da737734b9e77a5bb0ab3b5773cd93fde32846bd
Default Java Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/4/testReport/
Max. process+thread count 1955 (vs. ulimit of 5500)
modules C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/4/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 1m 0s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 1 new or modified test files.
_ fgl Compile Tests _
+0 🆗 mvndep 12m 35s Maven dependency ordering for branch
+1 💚 mvninstall 26m 11s fgl passed
+1 💚 compile 26m 2s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 compile 20m 52s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 checkstyle 4m 24s fgl passed
+1 💚 mvnsite 3m 41s fgl passed
+1 💚 javadoc 2m 20s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 25s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ spotbugs 2m 34s /branch-spotbugs-hadoop-common-project_hadoop-common-warnings.html hadoop-common-project/hadoop-common in fgl has 2 extant spotbugs warnings.
+1 💚 shadedclient 21m 20s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 25s Maven dependency ordering for patch
+1 💚 mvninstall 2m 31s the patch passed
+1 💚 compile 25m 2s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
-1 ❌ javac 25m 2s /results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1917 unchanged - 1 fixed = 1918 total (was 1918)
+1 💚 compile 20m 38s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ javac 20m 38s /results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu120.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu120.04-b10 generated 1 new + 1793 unchanged - 1 fixed = 1794 total (was 1794)
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 4m 4s /results-checkstyle-root.txt root: The patch generated 42 new + 291 unchanged - 0 fixed = 333 total (was 291)
+1 💚 mvnsite 3m 19s the patch passed
+1 💚 xml 0m 2s The patch has no ill-formed XML file.
+1 💚 javadoc 2m 11s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 12s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 spotbugs 7m 7s the patch passed
+1 💚 shadedclient 23m 20s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 18m 10s hadoop-common in the patch passed.
-1 ❌ unit 501m 48s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 1m 26s The patch does not generate ASF License warnings.
739m 17s
Reason Tests
Failed junit tests hadoop.hdfs.server.namenode.snapshot.TestCheckpointsWithSnapshots
hadoop.hdfs.server.namenode.TestStorageRestore
hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics
hadoop.hdfs.server.blockmanagement.TestReplicationPolicy
hadoop.hdfs.TestDFSUpgradeFromImage
hadoop.hdfs.TestErasureCodeBenchmarkThroughput
hadoop.hdfs.server.namenode.TestSecondaryNameNodeUpgrade
hadoop.hdfs.TestDFSShell
hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.TestRollingUpgradeDowngrade
hadoop.hdfs.server.namenode.snapshot.TestFSImageWithOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.TestFSNamesystem
hadoop.hdfs.server.namenode.snapshot.TestAclWithSnapshot
hadoop.hdfs.server.namenode.TestFSImage
hadoop.hdfs.server.namenode.snapshot.TestSnapRootDescendantDiff
hadoop.hdfs.server.namenode.snapshot.TestRenameWithSnapshots
hadoop.hdfs.server.namenode.TestCommitBlockSynchronization
hadoop.hdfs.TestDistributedFileSystem
hadoop.hdfs.server.namenode.ha.TestEditLogTailer
hadoop.hdfs.server.namenode.TestStartup
hadoop.hdfs.server.namenode.TestNameEditsConfigs
hadoop.hdfs.server.namenode.snapshot.TestXAttrWithSnapshot
hadoop.hdfs.TestRollingUpgradeRollback
hadoop.hdfs.server.namenode.TestFSNamesystemLock
hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor
hadoop.hdfs.server.namenode.TestFSNamesystemLockReport
hadoop.hdfs.server.namenode.snapshot.TestOpenFilesWithSnapshot
hadoop.hdfs.server.namenode.TestFSImageWithSnapshot
hadoop.hdfs.server.namenode.TestNameNodeRecovery
hadoop.hdfs.TestErasureCodingPolicyWithSnapshot
hadoop.hdfs.server.namenode.ha.TestFailureToReadEdits
hadoop.hdfs.server.namenode.TestAuditLogs
hadoop.hdfs.TestViewDistributedFileSystem
hadoop.hdfs.server.blockmanagement.TestBlockTokenWithDFSStriped
hadoop.hdfs.TestErasureCodingPolicyWithSnapshotWithRandomECPolicy
hadoop.hdfs.server.namenode.TestDecommissioningStatus
hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.TestCacheDirectives
hadoop.hdfs.server.namenode.TestPersistentStoragePolicySatisfier
hadoop.hdfs.server.namenode.snapshot.TestSnapshot
hadoop.hdfs.server.namenode.snapshot.TestSnapshotBlocksMap
hadoop.hdfs.server.namenode.ha.TestSeveralNameNodes
hadoop.hdfs.server.namenode.TestDeleteRace
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
hadoop.hdfs.server.namenode.TestFileTruncate
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDiffReport
hadoop.hdfs.server.namenode.TestCacheDirectivesWithViewDFS
hadoop.hdfs.web.TestWebHdfsFileSystemContract
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/6/artifact/out/Dockerfile
GITHUB PR #3417
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux e1581f9a769c 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision fgl / 8dae87f5ea7b8d5b2eef3d2037ec6434ebe4fc3a
Default Java Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/6/testReport/
Max. process+thread count 2140 (vs. ulimit of 5500)
modules C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/6/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 1m 34s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 2s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 1 new or modified test files.
_ fgl Compile Tests _
+0 🆗 mvndep 12m 56s Maven dependency ordering for branch
+1 💚 mvninstall 23m 3s fgl passed
+1 💚 compile 23m 3s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 compile 19m 20s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 checkstyle 3m 49s fgl passed
+1 💚 mvnsite 3m 8s fgl passed
+1 💚 javadoc 2m 4s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 9s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ spotbugs 2m 28s /branch-spotbugs-hadoop-common-project_hadoop-common-warnings.html hadoop-common-project/hadoop-common in fgl has 2 extant spotbugs warnings.
+1 💚 shadedclient 21m 37s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 29s Maven dependency ordering for patch
+1 💚 mvninstall 2m 57s the patch passed
+1 💚 compile 32m 8s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
-1 ❌ javac 32m 8s /results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1914 unchanged - 1 fixed = 1915 total (was 1915)
+1 💚 compile 32m 5s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ javac 32m 5s /results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu120.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu120.04-b10 generated 1 new + 1790 unchanged - 1 fixed = 1791 total (was 1791)
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 5m 27s /results-checkstyle-root.txt root: The patch generated 44 new + 291 unchanged - 0 fixed = 335 total (was 291)
+1 💚 mvnsite 4m 51s the patch passed
+1 💚 xml 0m 2s The patch has no ill-formed XML file.
+1 💚 javadoc 2m 32s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 58s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 spotbugs 8m 52s the patch passed
+1 💚 shadedclient 25m 10s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 19m 47s /patch-unit-hadoop-common-project_hadoop-common.txt hadoop-common in the patch passed.
-1 ❌ unit 447m 8s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 1m 2s The patch does not generate ASF License warnings.
704m 40s
Reason Tests
Failed junit tests hadoop.metrics2.source.TestJvmMetrics
hadoop.fs.viewfs.TestViewFSOverloadSchemeWithMountTableConfigInHDFS
hadoop.hdfs.server.namenode.TestDeleteRace
hadoop.hdfs.server.namenode.snapshot.TestSnapshotBlocksMap
hadoop.hdfs.server.namenode.TestNameNodeRecovery
hadoop.hdfs.server.namenode.TestDecommissioningStatusWithBackoffMonitor
hadoop.hdfs.TestDFSUpgradeFromImage
hadoop.hdfs.server.datanode.TestBatchIbr
hadoop.hdfs.server.namenode.snapshot.TestSnapshot
hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.snapshot.TestRenameWithSnapshots
hadoop.hdfs.server.namenode.ha.TestEditLogTailer
hadoop.metrics2.sink.TestRollingFileSystemSinkWithHdfs
hadoop.hdfs.TestErasureCodingPolicyWithSnapshot
hadoop.hdfs.TestErasureCodingPolicyWithSnapshotWithRandomECPolicy
hadoop.hdfs.server.namenode.ha.TestFailureToReadEdits
hadoop.hdfs.TestViewDistributedFileSystem
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
hadoop.hdfs.server.namenode.TestFileTruncate
hadoop.cli.TestHDFSCLI
hadoop.hdfs.TestDFSShell
hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.server.namenode.TestStorageRestore
hadoop.hdfs.TestStateAlignmentContextWithHA
hadoop.hdfs.server.namenode.snapshot.TestCheckpointsWithSnapshots
hadoop.hdfs.server.namenode.ha.TestSeveralNameNodes
hadoop.hdfs.server.namenode.TestNameEditsConfigs
hadoop.hdfs.server.namenode.TestFSNamesystemLockReport
hadoop.hdfs.TestRollingUpgradeRollback
hadoop.hdfs.server.namenode.TestCacheDirectivesWithViewDFS
hadoop.hdfs.server.namenode.TestFSImageWithSnapshot
hadoop.hdfs.server.namenode.TestParallelImageWrite
hadoop.hdfs.server.namenode.TestCommitBlockSynchronization
hadoop.hdfs.server.namenode.TestNameNodeMXBean
hadoop.hdfs.server.namenode.TestPersistentStoragePolicySatisfier
hadoop.hdfs.server.namenode.snapshot.TestRandomOpsWithSnapshots
hadoop.hdfs.server.namenode.snapshot.TestOpenFilesWithSnapshot
hadoop.security.TestPermissionSymlinks
hadoop.hdfs.server.namenode.TestFSImage
hadoop.hdfs.server.namenode.snapshot.TestSnapRootDescendantDiff
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDiffReport
hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfierWithStripedFile
hadoop.hdfs.server.namenode.snapshot.TestXAttrWithSnapshot
hadoop.hdfs.server.namenode.snapshot.TestFSImageWithOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.ha.TestPipelinesFailover
hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeHdfsFileSystemContract
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDeletion
hadoop.hdfs.server.namenode.TestFSNamesystemLock
hadoop.hdfs.server.namenode.TestFSNamesystem
hadoop.hdfs.TestDistributedFileSystem
hadoop.hdfs.server.namenode.TestDecommissioningStatus
hadoop.hdfs.server.namenode.TestCacheDirectives
hadoop.hdfs.server.namenode.TestStartup
hadoop.hdfs.TestRollingUpgradeDowngrade
hadoop.hdfs.server.namenode.snapshot.TestAclWithSnapshot
hadoop.hdfs.server.namenode.TestSecondaryNameNodeUpgrade
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/7/artifact/out/Dockerfile
GITHUB PR #3417
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux 3308794443c4 4.15.0-153-generic #160-Ubuntu SMP Thu Jul 29 06:54:29 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision fgl / 76553d128b3c49999cf5e346cd5f446454e2a440
Default Java Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/7/testReport/
Max. process+thread count 2837 (vs. ulimit of 5500)
modules C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/7/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 42s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 1 new or modified test files.
_ fgl Compile Tests _
+0 🆗 mvndep 13m 12s Maven dependency ordering for branch
+1 💚 mvninstall 20m 36s fgl passed
+1 💚 compile 22m 20s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 compile 19m 13s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 checkstyle 3m 52s fgl passed
+1 💚 mvnsite 3m 10s fgl passed
+1 💚 javadoc 2m 10s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 17s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ spotbugs 2m 24s /branch-spotbugs-hadoop-common-project_hadoop-common-warnings.html hadoop-common-project/hadoop-common in fgl has 2 extant spotbugs warnings.
+1 💚 shadedclient 17m 19s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 28s Maven dependency ordering for patch
+1 💚 mvninstall 2m 15s the patch passed
+1 💚 compile 22m 57s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
-1 ❌ javac 22m 57s /results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1915 unchanged - 1 fixed = 1916 total (was 1916)
+1 💚 compile 20m 4s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ javac 20m 4s /results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu120.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu120.04-b10 generated 1 new + 1791 unchanged - 1 fixed = 1792 total (was 1792)
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 3m 32s /results-checkstyle-root.txt root: The patch generated 1 new + 291 unchanged - 0 fixed = 292 total (was 291)
+1 💚 mvnsite 3m 9s the patch passed
+1 💚 xml 0m 1s The patch has no ill-formed XML file.
+1 💚 javadoc 2m 12s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 25s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 spotbugs 6m 3s the patch passed
+1 💚 shadedclient 17m 7s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 17m 42s hadoop-common in the patch passed.
-1 ❌ unit 309m 26s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 1m 15s The patch does not generate ASF License warnings.
520m 35s
Reason Tests
Failed junit tests hadoop.hdfs.server.namenode.TestAuditLogs
hadoop.hdfs.server.namenode.snapshot.TestCheckpointsWithSnapshots
hadoop.hdfs.server.namenode.snapshot.TestFSImageWithOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.ha.TestPipelinesFailover
hadoop.fs.viewfs.TestViewFileSystemOverloadSchemeHdfsFileSystemContract
hadoop.hdfs.server.namenode.snapshot.TestRandomOpsWithSnapshots
hadoop.hdfs.TestFileChecksum
hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints
hadoop.hdfs.server.namenode.TestEditLogRace
hadoop.hdfs.server.namenode.TestFSNamesystemLock
hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.server.namenode.TestNameNodeRecovery
hadoop.hdfs.server.namenode.TestStartup
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
hadoop.hdfs.server.namenode.snapshot.TestOpenFilesWithSnapshot
hadoop.hdfs.server.namenode.TestFSImage
hadoop.hdfs.TestDFSUpgradeFromImage
hadoop.hdfs.TestViewDistributedFileSystem
hadoop.hdfs.server.namenode.TestStorageRestore
hadoop.hdfs.TestRollingUpgradeDowngrade
hadoop.hdfs.server.namenode.TestFSNamesystem
hadoop.hdfs.server.namenode.snapshot.TestSnapRootDescendantDiff
hadoop.hdfs.server.namenode.TestPersistentStoragePolicySatisfier
hadoop.hdfs.server.namenode.TestFSNamesystemLockReport
hadoop.hdfs.server.datanode.TestBatchIbr
hadoop.hdfs.server.namenode.ha.TestFailureToReadEdits
hadoop.hdfs.server.namenode.snapshot.TestSnapshotBlocksMap
hadoop.hdfs.server.namenode.TestNameEditsConfigs
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDiffReport
hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDeletion
hadoop.hdfs.server.namenode.TestCacheDirectivesWithViewDFS
hadoop.hdfs.server.namenode.snapshot.TestAclWithSnapshot
hadoop.hdfs.TestRollingUpgradeRollback
hadoop.hdfs.TestErasureCodingPolicyWithSnapshot
hadoop.hdfs.TestDistributedFileSystem
hadoop.hdfs.server.namenode.TestFSImageWithSnapshot
hadoop.hdfs.server.namenode.TestSecondaryNameNodeUpgrade
hadoop.hdfs.server.namenode.snapshot.TestSnapshot
hadoop.hdfs.server.namenode.snapshot.TestRenameWithSnapshots
hadoop.hdfs.server.namenode.TestFileTruncate
hadoop.hdfs.server.namenode.ha.TestObserverNode
hadoop.hdfs.server.namenode.TestCacheDirectives
hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics
hadoop.hdfs.server.namenode.snapshot.TestXAttrWithSnapshot
hadoop.hdfs.TestErasureCodingPolicyWithSnapshotWithRandomECPolicy
hadoop.hdfs.server.namenode.TestCommitBlockSynchronization
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/8/artifact/out/Dockerfile
GITHUB PR #3417
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux 32c42df49910 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision fgl / 728ef528374258f4188d926bbabbc1f8967f980d
Default Java Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/8/testReport/
Max. process+thread count 3181 (vs. ulimit of 5500)
modules C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/8/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@jianghuazhu
Copy link
Contributor Author

@xinglin, can you help review?
There are some spotbugs tips in jenkins, for example:
This code increments a volatile field. Increments of volatile fields aren't atomic. If more than one thread is incrementing the field at the same time, increments could be lost.
In response to this prompt, I created another jira to respond:
https://issues.apache.org/jira/browse/HDFS-16228

Here are some phenomena in the debugging process.
When numSpaceKeyDepth=2, numRanges=10
{INodeDirectory} "range key: [0, 16385]"
{INodeDirectory} "range key: [1, 16385]"
{INodeDirectory} "range key: [2, 16385]"
{INodeDirectory} "range key: [3, 16385]"
{INodeDirectory} "range key: [4, 16385]"
{INodeDirectory} "range key: [5, 16385]"
{INodeDirectory} "range key: [6, 16385]"
{INodeDirectory} "range key: [7, 16385]"
{INodeDirectory} "range key: [8, 16385]"
{INodeDirectory} "range key: [9, 16385]"
https://issues.apache.org/jira/secure/attachment/13033627/13033627_debug1.jpg

When numSpaceKeyDepth=4, numRanges=10
{INodeDirectory} "range key: [0, 0, 0,16385]"
{INodeDirectory} "range key: [1, 0, 0,16385]"
{INodeDirectory} "range key: [2, 0, 0,16385]"
{INodeDirectory} "range key: [3, 0, 0,16385]"
{INodeDirectory} "range key: [4, 0, 0,16385]"
{INodeDirectory} "range key: [5, 0, 0,16385]"
{INodeDirectory} "range key: [6, 0, 0,16385]"
{INodeDirectory} "range key: [7, 0, 0,16385]"
{INodeDirectory} "range key: [8, 0, 0,16385]"
{INodeDirectory} "range key: [9, 0, 0,16385]"
https://issues.apache.org/jira/secure/attachment/13033628/13033628_debug2.jpg

Copy link
Contributor

@xinglin xinglin left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM!

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add a check here to make sure numRanges is power of 2? thanks,

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @xinglin for the comment.
I will update it later.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 1m 6s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 1 new or modified test files.
_ fgl Compile Tests _
+0 🆗 mvndep 12m 40s Maven dependency ordering for branch
+1 💚 mvninstall 23m 30s fgl passed
+1 💚 compile 23m 7s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 compile 19m 48s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 checkstyle 3m 58s fgl passed
+1 💚 mvnsite 3m 7s fgl passed
+1 💚 javadoc 2m 10s fgl passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 10s fgl passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ spotbugs 2m 27s /branch-spotbugs-hadoop-common-project_hadoop-common-warnings.html hadoop-common-project/hadoop-common in fgl has 2 extant spotbugs warnings.
+1 💚 shadedclient 19m 31s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 🆗 mvndep 0m 24s Maven dependency ordering for patch
+1 💚 mvninstall 2m 14s the patch passed
+1 💚 compile 22m 17s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
-1 ❌ javac 22m 17s /results-compile-javac-root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt root-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 1915 unchanged - 1 fixed = 1916 total (was 1916)
+1 💚 compile 19m 34s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
-1 ❌ javac 19m 34s /results-compile-javac-root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt root-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu120.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu120.04-b10 generated 1 new + 1791 unchanged - 1 fixed = 1792 total (was 1792)
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 3m 51s /results-checkstyle-root.txt root: The patch generated 1 new + 291 unchanged - 0 fixed = 292 total (was 291)
+1 💚 mvnsite 3m 5s the patch passed
+1 💚 xml 0m 2s The patch has no ill-formed XML file.
+1 💚 javadoc 2m 5s the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04
+1 💚 javadoc 3m 14s the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
+1 💚 spotbugs 6m 10s the patch passed
+1 💚 shadedclient 19m 14s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 16m 53s hadoop-common in the patch passed.
-1 ❌ unit 403m 42s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 💚 asflicense 1m 4s The patch does not generate ASF License warnings.
620m 30s
Reason Tests
Failed junit tests hadoop.hdfs.server.namenode.snapshot.TestCheckpointsWithSnapshots
hadoop.hdfs.server.namenode.TestStorageRestore
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDeletion
hadoop.hdfs.server.namenode.metrics.TestNameNodeMetrics
hadoop.hdfs.server.namenode.ha.TestPipelinesFailover
hadoop.hdfs.server.blockmanagement.TestReplicationPolicy
hadoop.hdfs.TestDFSUpgradeFromImage
hadoop.hdfs.server.namenode.TestSecondaryNameNodeUpgrade
hadoop.hdfs.TestDFSShell
hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.TestRollingUpgradeDowngrade
hadoop.hdfs.server.namenode.snapshot.TestFSImageWithOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.TestFSNamesystem
hadoop.hdfs.server.namenode.snapshot.TestAclWithSnapshot
hadoop.hdfs.server.namenode.TestFSImage
hadoop.hdfs.server.namenode.snapshot.TestSnapRootDescendantDiff
hadoop.hdfs.server.namenode.snapshot.TestRenameWithSnapshots
hadoop.hdfs.server.namenode.TestCommitBlockSynchronization
hadoop.hdfs.TestDistributedFileSystem
hadoop.hdfs.server.namenode.ha.TestEditLogTailer
hadoop.hdfs.server.namenode.TestStartup
hadoop.hdfs.server.namenode.TestNameEditsConfigs
hadoop.hdfs.server.namenode.snapshot.TestXAttrWithSnapshot
hadoop.hdfs.TestRollingUpgradeRollback
hadoop.hdfs.server.namenode.TestFSNamesystemLock
hadoop.hdfs.TestParallelRead
hadoop.hdfs.server.namenode.TestFSNamesystemLockReport
hadoop.hdfs.server.namenode.snapshot.TestOpenFilesWithSnapshot
hadoop.hdfs.server.balancer.TestBalancer
hadoop.hdfs.server.namenode.TestFSImageWithSnapshot
hadoop.hdfs.server.namenode.TestNameNodeRecovery
hadoop.hdfs.TestErasureCodingPolicyWithSnapshot
hadoop.hdfs.server.namenode.ha.TestFailureToReadEdits
hadoop.hdfs.server.namenode.TestAuditLogs
hadoop.hdfs.TestViewDistributedFileSystem
hadoop.hdfs.TestErasureCodingPolicyWithSnapshotWithRandomECPolicy
hadoop.hdfs.server.namenode.TestDecommissioningStatus
hadoop.hdfs.TestParallelShortCircuitReadUnCached
hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion
hadoop.hdfs.server.namenode.TestCacheDirectives
hadoop.hdfs.server.namenode.snapshot.TestSnapshot
hadoop.hdfs.server.datanode.TestBatchIbr
hadoop.hdfs.server.namenode.snapshot.TestSnapshotBlocksMap
hadoop.hdfs.server.namenode.TestDeleteRace
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
hadoop.hdfs.server.namenode.TestFileTruncate
hadoop.hdfs.server.namenode.snapshot.TestSnapshotDiffReport
hadoop.hdfs.server.namenode.TestCacheDirectivesWithViewDFS
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/9/artifact/out/Dockerfile
GITHUB PR #3417
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell xml
uname Linux 7e0e01c7d618 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision fgl / 40d3186
Default Java Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/9/testReport/
Max. process+thread count 3143 (vs. ulimit of 5500)
modules C: hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3417/9/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org

This message was automatically generated.

@jianghuazhu
Copy link
Contributor Author

@shvachko Could you please review this PR. Thanks.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants