|
@@ -18,6 +18,8 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
+import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
|
|
|
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
@@ -26,6 +28,8 @@ import static org.junit.Assert.fail;
|
|
|
import java.io.IOException;
|
|
|
import java.util.List;
|
|
|
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
@@ -55,12 +59,14 @@ import com.google.common.collect.Lists;
|
|
|
* Tests to verify safe mode correctness.
|
|
|
*/
|
|
|
public class TestSafeMode {
|
|
|
+ public static final Log LOG = LogFactory.getLog(TestSafeMode.class);
|
|
|
private static final Path TEST_PATH = new Path("/test");
|
|
|
private static final int BLOCK_SIZE = 1024;
|
|
|
Configuration conf;
|
|
|
MiniDFSCluster cluster;
|
|
|
FileSystem fs;
|
|
|
DistributedFileSystem dfs;
|
|
|
+ private static final String NN_METRICS = "NameNodeActivity";
|
|
|
|
|
|
@Before
|
|
|
public void startUp() throws IOException {
|
|
@@ -158,6 +164,7 @@ public class TestSafeMode {
|
|
|
*/
|
|
|
@Test(timeout=45000)
|
|
|
public void testInitializeReplQueuesEarly() throws Exception {
|
|
|
+ LOG.info("Starting testInitializeReplQueuesEarly");
|
|
|
// Spray the blocks around the cluster when we add DNs instead of
|
|
|
// concentrating all blocks on the first node.
|
|
|
BlockManagerTestUtil.setWritingPrefersLocalNode(
|
|
@@ -165,9 +172,11 @@ public class TestSafeMode {
|
|
|
|
|
|
cluster.startDataNodes(conf, 2, true, StartupOption.REGULAR, null);
|
|
|
cluster.waitActive();
|
|
|
+
|
|
|
+ LOG.info("Creating files");
|
|
|
DFSTestUtil.createFile(fs, TEST_PATH, 15*BLOCK_SIZE, (short)1, 1L);
|
|
|
|
|
|
-
|
|
|
+ LOG.info("Stopping all DataNodes");
|
|
|
List<DataNodeProperties> dnprops = Lists.newLinkedList();
|
|
|
dnprops.add(cluster.stopDataNode(0));
|
|
|
dnprops.add(cluster.stopDataNode(0));
|
|
@@ -176,6 +185,7 @@ public class TestSafeMode {
|
|
|
cluster.getConfiguration(0).setFloat(
|
|
|
DFSConfigKeys.DFS_NAMENODE_REPL_QUEUE_THRESHOLD_PCT_KEY, 1f/15f);
|
|
|
|
|
|
+ LOG.info("Restarting NameNode");
|
|
|
cluster.restartNameNode();
|
|
|
final NameNode nn = cluster.getNameNode();
|
|
|
|
|
@@ -189,27 +199,37 @@ public class TestSafeMode {
|
|
|
"until threshold is crossed",
|
|
|
NameNodeAdapter.safeModeInitializedReplQueues(nn));
|
|
|
|
|
|
+ LOG.info("Restarting one DataNode");
|
|
|
cluster.restartDataNode(dnprops.remove(0));
|
|
|
|
|
|
- // Wait for the block report from the restarted DN to come in.
|
|
|
+ // Wait for block reports from all attached storages of
|
|
|
+ // the restarted DN to come in.
|
|
|
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
@Override
|
|
|
public Boolean get() {
|
|
|
- return NameNodeAdapter.getSafeModeSafeBlocks(nn) > 0;
|
|
|
+ return getLongCounter("StorageBlockReportOps", getMetrics(NN_METRICS)) ==
|
|
|
+ MiniDFSCluster.DIRS_PER_DATANODE;
|
|
|
}
|
|
|
}, 10, 10000);
|
|
|
- // SafeMode is fine-grain synchronized, so the processMisReplicatedBlocks
|
|
|
- // call is still going on at this point - wait until it's done by grabbing
|
|
|
- // the lock.
|
|
|
- nn.getNamesystem().writeLock();
|
|
|
- nn.getNamesystem().writeUnlock();
|
|
|
- int safe = NameNodeAdapter.getSafeModeSafeBlocks(nn);
|
|
|
- assertTrue("Expected first block report to make some but not all blocks " +
|
|
|
- "safe. Got: " + safe, safe >= 1 && safe < 15);
|
|
|
- BlockManagerTestUtil.updateState(nn.getNamesystem().getBlockManager());
|
|
|
-
|
|
|
+
|
|
|
+ final int safe = NameNodeAdapter.getSafeModeSafeBlocks(nn);
|
|
|
+ assertTrue("Expected first block report to make some blocks safe.", safe > 0);
|
|
|
+ assertTrue("Did not expect first block report to make all blocks safe.", safe < 15);
|
|
|
+
|
|
|
assertTrue(NameNodeAdapter.safeModeInitializedReplQueues(nn));
|
|
|
- assertEquals(15 - safe, nn.getNamesystem().getUnderReplicatedBlocks());
|
|
|
+
|
|
|
+ // Ensure that UnderReplicatedBlocks goes up to 15 - safe. Misreplicated
|
|
|
+ // blocks are processed asynchronously so this may take a few seconds.
|
|
|
+ // Failure here will manifest as a test timeout.
|
|
|
+ BlockManagerTestUtil.updateState(nn.getNamesystem().getBlockManager());
|
|
|
+ long underReplicatedBlocks = nn.getNamesystem().getUnderReplicatedBlocks();
|
|
|
+ while (underReplicatedBlocks != (15 - safe)) {
|
|
|
+ LOG.info("UnderReplicatedBlocks expected=" + (15 - safe) +
|
|
|
+ ", actual=" + underReplicatedBlocks);
|
|
|
+ Thread.sleep(100);
|
|
|
+ BlockManagerTestUtil.updateState(nn.getNamesystem().getBlockManager());
|
|
|
+ underReplicatedBlocks = nn.getNamesystem().getUnderReplicatedBlocks();
|
|
|
+ }
|
|
|
|
|
|
cluster.restartDataNodes();
|
|
|
}
|