Skip to content

Commit 7bd7725

Browse files
authored
HDFS-16553. Fix checkstyle for the length of BlockManager construction method over limit. (#4211). Contributed by Chengwei Wang.
1 parent b9ade7a commit 7bd7725

File tree

1 file changed

+94
-66
lines changed
  • hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement

1 file changed

+94
-66
lines changed

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

Lines changed: 94 additions & 66 deletions
Original file line numberDiff line numberDiff line change
@@ -485,72 +485,54 @@ public int getPendingSPSPaths() {
485485
public BlockManager(final Namesystem namesystem, boolean haEnabled,
486486
final Configuration conf) throws IOException {
487487
this.namesystem = namesystem;
488-
datanodeManager = new DatanodeManager(this, namesystem, conf);
489-
heartbeatManager = datanodeManager.getHeartbeatManager();
488+
this.datanodeManager = new DatanodeManager(this, namesystem, conf);
489+
this.heartbeatManager = datanodeManager.getHeartbeatManager();
490490
this.blockIdManager = new BlockIdManager(this);
491-
blocksPerPostpondedRescan = (int)Math.min(Integer.MAX_VALUE,
491+
this.blocksPerPostpondedRescan = (int)Math.min(Integer.MAX_VALUE,
492492
datanodeManager.getBlocksPerPostponedMisreplicatedBlocksRescan());
493-
rescannedMisreplicatedBlocks =
493+
this.rescannedMisreplicatedBlocks =
494494
new ArrayList<Block>(blocksPerPostpondedRescan);
495-
startupDelayBlockDeletionInMs = conf.getLong(
495+
this.startupDelayBlockDeletionInMs = conf.getLong(
496496
DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_KEY,
497-
DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_DEFAULT) * 1000L;
498-
deleteBlockLockTimeMs = conf.getLong(
497+
DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_DEFAULT)
498+
* 1000L;
499+
this.deleteBlockLockTimeMs = conf.getLong(
499500
DFSConfigKeys.DFS_NAMENODE_BLOCK_DELETION_LOCK_THRESHOLD_MS,
500501
DFSConfigKeys.DFS_NAMENODE_BLOCK_DELETION_LOCK_THRESHOLD_MS_DEFAULT);
501-
deleteBlockUnlockIntervalTimeMs = conf.getLong(
502+
this.deleteBlockUnlockIntervalTimeMs = conf.getLong(
502503
DFSConfigKeys.DFS_NAMENODE_BLOCK_DELETION_UNLOCK_INTERVAL_MS,
503504
DFSConfigKeys.DFS_NAMENODE_BLOCK_DELETION_UNLOCK_INTERVAL_MS_DEFAULT);
504-
invalidateBlocks = new InvalidateBlocks(
505+
this.invalidateBlocks = new InvalidateBlocks(
505506
datanodeManager.getBlockInvalidateLimit(),
506507
startupDelayBlockDeletionInMs,
507508
blockIdManager);
508-
markedDeleteQueue = new ConcurrentLinkedQueue<>();
509+
this.markedDeleteQueue = new ConcurrentLinkedQueue<>();
509510
// Compute the map capacity by allocating 2% of total memory
510-
blocksMap = new BlocksMap(
511+
this.blocksMap = new BlocksMap(
511512
LightWeightGSet.computeCapacity(2.0, "BlocksMap"));
512-
placementPolicies = new BlockPlacementPolicies(
513-
conf, datanodeManager.getFSClusterStats(),
514-
datanodeManager.getNetworkTopology(),
515-
datanodeManager.getHost2DatanodeMap());
516-
storagePolicySuite = BlockStoragePolicySuite.createDefaultSuite(conf);
517-
pendingReconstruction = new PendingReconstructionBlocks(conf.getInt(
513+
this.placementPolicies = new BlockPlacementPolicies(
514+
conf, datanodeManager.getFSClusterStats(),
515+
datanodeManager.getNetworkTopology(),
516+
datanodeManager.getHost2DatanodeMap());
517+
this.storagePolicySuite = BlockStoragePolicySuite.createDefaultSuite(conf);
518+
this.pendingReconstruction = new PendingReconstructionBlocks(conf.getInt(
518519
DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY,
519520
DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT)
520521
* 1000L);
521522

522523
createSPSManager(conf);
523524

524-
blockTokenSecretManager = createBlockTokenSecretManager(conf);
525-
526-
providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);
525+
this.blockTokenSecretManager = createBlockTokenSecretManager(conf);
526+
this.providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);
527527

528528
this.maxCorruptFilesReturned = conf.getInt(
529-
DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY,
530-
DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED);
529+
DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY,
530+
DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED);
531531
this.defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
532532
DFSConfigKeys.DFS_REPLICATION_DEFAULT);
533533

534-
final int maxR = conf.getInt(DFSConfigKeys.DFS_REPLICATION_MAX_KEY,
535-
DFSConfigKeys.DFS_REPLICATION_MAX_DEFAULT);
536-
final int minR = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
537-
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
538-
if (minR <= 0)
539-
throw new IOException("Unexpected configuration parameters: "
540-
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY
541-
+ " = " + minR + " <= 0");
542-
if (maxR > Short.MAX_VALUE)
543-
throw new IOException("Unexpected configuration parameters: "
544-
+ DFSConfigKeys.DFS_REPLICATION_MAX_KEY
545-
+ " = " + maxR + " > " + Short.MAX_VALUE);
546-
if (minR > maxR)
547-
throw new IOException("Unexpected configuration parameters: "
548-
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY
549-
+ " = " + minR + " > "
550-
+ DFSConfigKeys.DFS_REPLICATION_MAX_KEY
551-
+ " = " + maxR);
552-
this.minReplication = (short)minR;
553-
this.maxReplication = (short)maxR;
534+
this.minReplication = (short) initMinReplication(conf);
535+
this.maxReplication = (short) initMaxReplication(conf);
554536

555537
this.maxReplicationStreams =
556538
conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY,
@@ -582,6 +564,66 @@ public BlockManager(final Namesystem namesystem, boolean haEnabled,
582564
DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT,
583565
DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT_DEFAULT);
584566

567+
this.minReplicationToBeInMaintenance =
568+
(short) initMinReplicationToBeInMaintenance(conf);
569+
this.replQueueResetToHeadThreshold =
570+
initReplQueueResetToHeadThreshold(conf);
571+
572+
long heartbeatIntervalSecs = conf.getTimeDuration(
573+
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
574+
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
575+
long blockRecoveryTimeout = getBlockRecoveryTimeout(heartbeatIntervalSecs);
576+
this.pendingRecoveryBlocks = new PendingRecoveryBlocks(blockRecoveryTimeout);
577+
578+
this.blockReportLeaseManager = new BlockReportLeaseManager(conf);
579+
580+
this.bmSafeMode = new BlockManagerSafeMode(this, namesystem, haEnabled,
581+
conf);
582+
583+
int queueSize = conf.getInt(
584+
DFSConfigKeys.DFS_NAMENODE_BLOCKREPORT_QUEUE_SIZE_KEY,
585+
DFSConfigKeys.DFS_NAMENODE_BLOCKREPORT_QUEUE_SIZE_DEFAULT);
586+
this.blockReportThread = new BlockReportProcessingThread(queueSize);
587+
588+
this.deleteCorruptReplicaImmediately =
589+
conf.getBoolean(DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED,
590+
DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED_DEFAULT);
591+
592+
printInitialConfigs();
593+
}
594+
595+
private int initMinReplication(Configuration conf) throws IOException {
596+
final int minR = conf.getInt(
597+
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
598+
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
599+
if (minR <= 0) {
600+
throw new IOException("Unexpected configuration parameters: "
601+
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY
602+
+ " = " + minR + " <= 0");
603+
}
604+
return minR;
605+
}
606+
607+
private int initMaxReplication(Configuration conf) throws IOException {
608+
final int maxR = conf.getInt(DFSConfigKeys.DFS_REPLICATION_MAX_KEY,
609+
DFSConfigKeys.DFS_REPLICATION_MAX_DEFAULT);
610+
if (maxR > Short.MAX_VALUE) {
611+
throw new IOException("Unexpected configuration parameters: "
612+
+ DFSConfigKeys.DFS_REPLICATION_MAX_KEY
613+
+ " = " + maxR + " > " + Short.MAX_VALUE);
614+
}
615+
if (minReplication > maxR) {
616+
throw new IOException("Unexpected configuration parameters: "
617+
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY
618+
+ " = " + minReplication + " > "
619+
+ DFSConfigKeys.DFS_REPLICATION_MAX_KEY
620+
+ " = " + maxR);
621+
}
622+
return maxR;
623+
}
624+
625+
private int initMinReplicationToBeInMaintenance(Configuration conf)
626+
throws IOException {
585627
final int minMaintenanceR = conf.getInt(
586628
DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY,
587629
DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_DEFAULT);
@@ -598,39 +640,25 @@ public BlockManager(final Namesystem namesystem, boolean haEnabled,
598640
+ DFSConfigKeys.DFS_REPLICATION_KEY
599641
+ " = " + defaultReplication);
600642
}
601-
this.minReplicationToBeInMaintenance = (short)minMaintenanceR;
643+
return minMaintenanceR;
644+
}
602645

603-
replQueueResetToHeadThreshold = conf.getInt(
646+
private int initReplQueueResetToHeadThreshold(Configuration conf) {
647+
int threshold = conf.getInt(
604648
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_QUEUE_RESTART_ITERATIONS,
605649
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_QUEUE_RESTART_ITERATIONS_DEFAULT);
606-
if (replQueueResetToHeadThreshold < 0) {
650+
if (threshold < 0) {
607651
LOG.warn("{} is set to {} and it must be >= 0. Resetting to default {}",
608652
DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_QUEUE_RESTART_ITERATIONS,
609-
replQueueResetToHeadThreshold, DFSConfigKeys.
653+
threshold, DFSConfigKeys.
610654
DFS_NAMENODE_REDUNDANCY_QUEUE_RESTART_ITERATIONS_DEFAULT);
611-
replQueueResetToHeadThreshold = DFSConfigKeys.
655+
threshold = DFSConfigKeys.
612656
DFS_NAMENODE_REDUNDANCY_QUEUE_RESTART_ITERATIONS_DEFAULT;
613657
}
658+
return threshold;
659+
}
614660

615-
long heartbeatIntervalSecs = conf.getTimeDuration(
616-
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
617-
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
618-
long blockRecoveryTimeout = getBlockRecoveryTimeout(heartbeatIntervalSecs);
619-
pendingRecoveryBlocks = new PendingRecoveryBlocks(blockRecoveryTimeout);
620-
621-
this.blockReportLeaseManager = new BlockReportLeaseManager(conf);
622-
623-
bmSafeMode = new BlockManagerSafeMode(this, namesystem, haEnabled, conf);
624-
625-
int queueSize = conf.getInt(
626-
DFSConfigKeys.DFS_NAMENODE_BLOCKREPORT_QUEUE_SIZE_KEY,
627-
DFSConfigKeys.DFS_NAMENODE_BLOCKREPORT_QUEUE_SIZE_DEFAULT);
628-
blockReportThread = new BlockReportProcessingThread(queueSize);
629-
630-
this.deleteCorruptReplicaImmediately =
631-
conf.getBoolean(DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED,
632-
DFS_NAMENODE_CORRUPT_BLOCK_DELETE_IMMEDIATELY_ENABLED_DEFAULT);
633-
661+
private void printInitialConfigs() {
634662
LOG.info("defaultReplication = {}", defaultReplication);
635663
LOG.info("maxReplication = {}", maxReplication);
636664
LOG.info("minReplication = {}", minReplication);

0 commit comments

Comments
 (0)