|
@@ -72,6 +72,7 @@ import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
|
|
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
|
|
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
|
|
@@ -127,6 +128,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
private volatile long corruptReplicaBlocksCount = 0L;
|
|
private volatile long corruptReplicaBlocksCount = 0L;
|
|
private volatile long underReplicatedBlocksCount = 0L;
|
|
private volatile long underReplicatedBlocksCount = 0L;
|
|
private volatile long scheduledReplicationBlocksCount = 0L;
|
|
private volatile long scheduledReplicationBlocksCount = 0L;
|
|
|
|
+
|
|
|
|
+ /** flag indicating whether replication queues have been initialized */
|
|
|
|
+ private boolean initializedReplQueues;
|
|
|
|
+
|
|
private final AtomicLong excessBlocksCount = new AtomicLong(0L);
|
|
private final AtomicLong excessBlocksCount = new AtomicLong(0L);
|
|
private final AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
|
|
private final AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
|
|
private final long startupDelayBlockDeletionInMs;
|
|
private final long startupDelayBlockDeletionInMs;
|
|
@@ -1092,7 +1097,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
* datanode and log the operation
|
|
* datanode and log the operation
|
|
*/
|
|
*/
|
|
void addToInvalidates(final Block block, final DatanodeInfo datanode) {
|
|
void addToInvalidates(final Block block, final DatanodeInfo datanode) {
|
|
- if (!namesystem.isPopulatingReplQueues()) {
|
|
|
|
|
|
+ if (!isPopulatingReplQueues()) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
invalidateBlocks.add(block, datanode, true);
|
|
invalidateBlocks.add(block, datanode, true);
|
|
@@ -1103,7 +1108,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
* datanodes.
|
|
* datanodes.
|
|
*/
|
|
*/
|
|
private void addToInvalidates(Block b) {
|
|
private void addToInvalidates(Block b) {
|
|
- if (!namesystem.isPopulatingReplQueues()) {
|
|
|
|
|
|
+ if (!isPopulatingReplQueues()) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
StringBuilder datanodes = new StringBuilder();
|
|
StringBuilder datanodes = new StringBuilder();
|
|
@@ -1124,7 +1129,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
* is wiped.
|
|
* is wiped.
|
|
*/
|
|
*/
|
|
void removeFromInvalidates(final DatanodeInfo datanode) {
|
|
void removeFromInvalidates(final DatanodeInfo datanode) {
|
|
- if (!namesystem.isPopulatingReplQueues()) {
|
|
|
|
|
|
+ if (!isPopulatingReplQueues()) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
invalidateBlocks.remove(datanode);
|
|
invalidateBlocks.remove(datanode);
|
|
@@ -1211,7 +1216,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|| corruptedDuringWrite) {
|
|
|| corruptedDuringWrite) {
|
|
// the block is over-replicated so invalidate the replicas immediately
|
|
// the block is over-replicated so invalidate the replicas immediately
|
|
invalidateBlock(b, node);
|
|
invalidateBlock(b, node);
|
|
- } else if (namesystem.isPopulatingReplQueues()) {
|
|
|
|
|
|
+ } else if (isPopulatingReplQueues()) {
|
|
// add the block to neededReplication
|
|
// add the block to neededReplication
|
|
updateNeededReplications(b.getStored(), -1, 0);
|
|
updateNeededReplications(b.getStored(), -1, 0);
|
|
}
|
|
}
|
|
@@ -2484,7 +2489,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
throws IOException {
|
|
throws IOException {
|
|
assert (storedBlock != null && namesystem.hasWriteLock());
|
|
assert (storedBlock != null && namesystem.hasWriteLock());
|
|
if (!namesystem.isInStartupSafeMode()
|
|
if (!namesystem.isInStartupSafeMode()
|
|
- || namesystem.isPopulatingReplQueues()) {
|
|
|
|
|
|
+ || isPopulatingReplQueues()) {
|
|
addStoredBlock(storedBlock, storageInfo, null, false);
|
|
addStoredBlock(storedBlock, storageInfo, null, false);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
@@ -2586,7 +2591,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
}
|
|
}
|
|
|
|
|
|
// do not try to handle over/under-replicated blocks during first safe mode
|
|
// do not try to handle over/under-replicated blocks during first safe mode
|
|
- if (!namesystem.isPopulatingReplQueues()) {
|
|
|
|
|
|
+ if (!isPopulatingReplQueues()) {
|
|
return storedBlock;
|
|
return storedBlock;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -3323,7 +3328,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
*/
|
|
*/
|
|
void processOverReplicatedBlocksOnReCommission(
|
|
void processOverReplicatedBlocksOnReCommission(
|
|
final DatanodeDescriptor srcNode) {
|
|
final DatanodeDescriptor srcNode) {
|
|
- if (!namesystem.isPopulatingReplQueues()) {
|
|
|
|
|
|
+ if (!isPopulatingReplQueues()) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
final Iterator<BlockInfo> it = srcNode.getBlockIterator();
|
|
final Iterator<BlockInfo> it = srcNode.getBlockIterator();
|
|
@@ -3417,7 +3422,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
final int curReplicasDelta, int expectedReplicasDelta) {
|
|
final int curReplicasDelta, int expectedReplicasDelta) {
|
|
namesystem.writeLock();
|
|
namesystem.writeLock();
|
|
try {
|
|
try {
|
|
- if (!namesystem.isPopulatingReplQueues()) {
|
|
|
|
|
|
+ if (!isPopulatingReplQueues()) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
NumberReplicas repl = countNodes(block);
|
|
NumberReplicas repl = countNodes(block);
|
|
@@ -3662,7 +3667,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
while (namesystem.isRunning()) {
|
|
while (namesystem.isRunning()) {
|
|
try {
|
|
try {
|
|
// Process replication work only when active NN is out of safe mode.
|
|
// Process replication work only when active NN is out of safe mode.
|
|
- if (namesystem.isPopulatingReplQueues()) {
|
|
|
|
|
|
+ if (isPopulatingReplQueues()) {
|
|
computeDatanodeWork();
|
|
computeDatanodeWork();
|
|
processPendingReplications();
|
|
processPendingReplications();
|
|
rescanPostponedMisreplicatedBlocks();
|
|
rescanPostponedMisreplicatedBlocks();
|
|
@@ -3790,4 +3795,35 @@ public class BlockManager implements BlockStatsMXBean {
|
|
public Map<StorageType, StorageTypeStats> getStorageTypeStats() {
|
|
public Map<StorageType, StorageTypeStats> getStorageTypeStats() {
|
|
return datanodeManager.getDatanodeStatistics().getStorageTypeStats();
|
|
return datanodeManager.getDatanodeStatistics().getStorageTypeStats();
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Initialize replication queues.
|
|
|
|
+ */
|
|
|
|
+ public void initializeReplQueues() {
|
|
|
|
+ LOG.info("initializing replication queues");
|
|
|
|
+ processMisReplicatedBlocks();
|
|
|
|
+ initializedReplQueues = true;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Check if replication queues are to be populated
|
|
|
|
+ * @return true when node is HAState.Active and not in the very first safemode
|
|
|
|
+ */
|
|
|
|
+ public boolean isPopulatingReplQueues() {
|
|
|
|
+ if (!shouldPopulateReplQueues()) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+ return initializedReplQueues;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void setInitializedReplQueues(boolean v) {
|
|
|
|
+ this.initializedReplQueues = v;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public boolean shouldPopulateReplQueues() {
|
|
|
|
+ HAContext haContext = namesystem.getHAContext();
|
|
|
|
+ if (haContext == null || haContext.getState() == null)
|
|
|
|
+ return false;
|
|
|
|
+ return haContext.getState().shouldPopulateReplQueues();
|
|
|
|
+ }
|
|
}
|
|
}
|