|
@@ -17,6 +17,10 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY;
|
|
|
+import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
|
|
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertNotNull;
|
|
|
import static org.junit.Assert.assertNull;
|
|
@@ -44,6 +48,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
|
|
|
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.Mockito;
|
|
|
|
|
@@ -178,7 +183,7 @@ public class TestPendingReconstruction {
|
|
|
public void testProcessPendingReconstructions() throws Exception {
|
|
|
final Configuration conf = new HdfsConfiguration();
|
|
|
conf.setLong(
|
|
|
- DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, TIMEOUT);
|
|
|
+ DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, TIMEOUT);
|
|
|
MiniDFSCluster cluster = null;
|
|
|
Block block;
|
|
|
BlockInfo blockInfo;
|
|
@@ -418,7 +423,7 @@ public class TestPendingReconstruction {
|
|
|
CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024);
|
|
|
CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
|
|
|
DFS_REPLICATION_INTERVAL);
|
|
|
- CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
|
|
|
+ CONF.setInt(DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
|
|
|
DFS_REPLICATION_INTERVAL);
|
|
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(
|
|
|
DATANODE_COUNT).build();
|
|
@@ -471,4 +476,81 @@ public class TestPendingReconstruction {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testReplicationCounter() throws Exception {
|
|
|
+ HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
+ conf.setInt(DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
|
|
|
+ conf.setInt(DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, 2);
|
|
|
+ MiniDFSCluster tmpCluster = new MiniDFSCluster.Builder(conf).numDataNodes(
|
|
|
+ DATANODE_COUNT).build();
|
|
|
+ tmpCluster.waitActive();
|
|
|
+ FSNamesystem fsn = tmpCluster.getNamesystem(0);
|
|
|
+ fsn.writeLock();
|
|
|
+
|
|
|
+ try {
|
|
|
+ BlockManager bm = fsn.getBlockManager();
|
|
|
+ BlocksMap blocksMap = bm.blocksMap;
|
|
|
+
|
|
|
+ // create three blockInfo below, blockInfo0 will success, blockInfo1 will
|
|
|
+ // time out, blockInfo2 will fail the replication.
|
|
|
+ BlockCollection bc0 = Mockito.mock(BlockCollection.class);
|
|
|
+ BlockInfo blockInfo0 = new BlockInfoContiguous((short) 3);
|
|
|
+ blockInfo0.setBlockId(0);
|
|
|
+
|
|
|
+ BlockCollection bc1 = Mockito.mock(BlockCollection.class);
|
|
|
+ BlockInfo blockInfo1 = new BlockInfoContiguous((short) 3);
|
|
|
+ blockInfo1.setBlockId(1);
|
|
|
+
|
|
|
+ BlockCollection bc2 = Mockito.mock(BlockCollection.class);
|
|
|
+ Mockito.when(bc2.getId()).thenReturn((2L));
|
|
|
+ BlockInfo blockInfo2 = new BlockInfoContiguous((short) 3);
|
|
|
+ blockInfo2.setBlockId(2);
|
|
|
+
|
|
|
+ blocksMap.addBlockCollection(blockInfo0, bc0);
|
|
|
+ blocksMap.addBlockCollection(blockInfo1, bc1);
|
|
|
+ blocksMap.addBlockCollection(blockInfo2, bc2);
|
|
|
+
|
|
|
+ PendingReconstructionBlocks pending = bm.pendingReconstruction;
|
|
|
+
|
|
|
+ MetricsRecordBuilder rb = getMetrics("NameNodeActivity");
|
|
|
+ assertCounter("SuccessfulReReplications", 0L, rb);
|
|
|
+ assertCounter("NumTimesReReplicationNotScheduled", 0L, rb);
|
|
|
+ assertCounter("TimeoutReReplications", 0L, rb);
|
|
|
+
|
|
|
+ // add block0 and block1 to pending queue.
|
|
|
+ pending.increment(blockInfo0);
|
|
|
+ pending.increment(blockInfo1);
|
|
|
+
|
|
|
+ Thread.sleep(2000);
|
|
|
+
|
|
|
+ rb = getMetrics("NameNodeActivity");
|
|
|
+ assertCounter("SuccessfulReReplications", 0L, rb);
|
|
|
+ assertCounter("NumTimesReReplicationNotScheduled", 0L, rb);
|
|
|
+ assertCounter("TimeoutReReplications", 0L, rb);
|
|
|
+
|
|
|
+ // call addBlock on block0 will make it successfully replicated.
|
|
|
+ // not callign addBlock on block1 will make it timeout later.
|
|
|
+ DatanodeStorageInfo[] storageInfos =
|
|
|
+ DFSTestUtil.createDatanodeStorageInfos(1);
|
|
|
+ bm.addBlock(storageInfos[0], blockInfo0, null);
|
|
|
+
|
|
|
+ // call schedule replication on blockInfo2 will fail the re-replication.
|
|
|
+ // because there is no source data to replicate from.
|
|
|
+ bm.scheduleReconstruction(blockInfo2, 0);
|
|
|
+
|
|
|
+ Thread.sleep(2000);
|
|
|
+
|
|
|
+ rb = getMetrics("NameNodeActivity");
|
|
|
+ assertCounter("SuccessfulReReplications", 1L, rb);
|
|
|
+ assertCounter("NumTimesReReplicationNotScheduled", 1L, rb);
|
|
|
+ assertCounter("TimeoutReReplications", 1L, rb);
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ tmpCluster.shutdown();
|
|
|
+ fsn.writeUnlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
}
|