|
@@ -19,22 +19,37 @@
|
|
package org.apache.hadoop.hdfs;
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.util.List;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
|
|
|
import static org.junit.Assert.*;
|
|
import static org.junit.Assert.*;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
|
|
|
|
+import com.google.common.base.Supplier;
|
|
|
|
+import com.google.common.collect.Lists;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Tests to verify safe mode correctness.
|
|
* Tests to verify safe mode correctness.
|
|
*/
|
|
*/
|
|
public class TestSafeMode {
|
|
public class TestSafeMode {
|
|
|
|
+ private static final Path TEST_PATH = new Path("/test");
|
|
|
|
+ private static final int BLOCK_SIZE = 1024;
|
|
Configuration conf;
|
|
Configuration conf;
|
|
MiniDFSCluster cluster;
|
|
MiniDFSCluster cluster;
|
|
FileSystem fs;
|
|
FileSystem fs;
|
|
@@ -43,6 +58,7 @@ public class TestSafeMode {
|
|
@Before
|
|
@Before
|
|
public void startUp() throws IOException {
|
|
public void startUp() throws IOException {
|
|
conf = new HdfsConfiguration();
|
|
conf = new HdfsConfiguration();
|
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
cluster.waitActive();
|
|
cluster.waitActive();
|
|
fs = cluster.getFileSystem();
|
|
fs = cluster.getFileSystem();
|
|
@@ -83,7 +99,7 @@ public class TestSafeMode {
|
|
|
|
|
|
// create two files with one block each.
|
|
// create two files with one block each.
|
|
DFSTestUtil.createFile(fs, file1, 1000, (short)1, 0);
|
|
DFSTestUtil.createFile(fs, file1, 1000, (short)1, 0);
|
|
- DFSTestUtil.createFile(fs, file2, 2000, (short)1, 0);
|
|
|
|
|
|
+ DFSTestUtil.createFile(fs, file2, 1000, (short)1, 0);
|
|
fs.close();
|
|
fs.close();
|
|
cluster.shutdown();
|
|
cluster.shutdown();
|
|
|
|
|
|
@@ -127,6 +143,106 @@ public class TestSafeMode {
|
|
String status = cluster.getNameNode().getNamesystem().getSafemode();
|
|
String status = cluster.getNameNode().getNamesystem().getSafemode();
|
|
assertEquals("", status);
|
|
assertEquals("", status);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test that the NN initializes its under-replicated blocks queue
|
|
|
|
+ * before it is ready to exit safemode (HDFS-1476)
|
|
|
|
+ */
|
|
|
|
+ @Test(timeout=45000)
|
|
|
|
+ public void testInitializeReplQueuesEarly() throws Exception {
|
|
|
|
+ // Spray the blocks around the cluster when we add DNs instead of
|
|
|
|
+ // concentrating all blocks on the first node.
|
|
|
|
+ BlockManagerTestUtil.setWritingPrefersLocalNode(
|
|
|
|
+ cluster.getNamesystem().getBlockManager(), false);
|
|
|
|
+
|
|
|
|
+ cluster.startDataNodes(conf, 2, true, StartupOption.REGULAR, null);
|
|
|
|
+ cluster.waitActive();
|
|
|
|
+ DFSTestUtil.createFile(fs, TEST_PATH, 15*BLOCK_SIZE, (short)1, 1L);
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ List<DataNodeProperties> dnprops = Lists.newLinkedList();
|
|
|
|
+ dnprops.add(cluster.stopDataNode(0));
|
|
|
|
+ dnprops.add(cluster.stopDataNode(0));
|
|
|
|
+ dnprops.add(cluster.stopDataNode(0));
|
|
|
|
+
|
|
|
|
+ cluster.getConfiguration(0).setFloat(
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_REPL_QUEUE_THRESHOLD_PCT_KEY, 1f/15f);
|
|
|
|
+
|
|
|
|
+ cluster.restartNameNode();
|
|
|
|
+ final NameNode nn = cluster.getNameNode();
|
|
|
|
+
|
|
|
|
+ String status = nn.getNamesystem().getSafemode();
|
|
|
|
+ assertEquals("Safe mode is ON.The reported blocks 0 needs additional " +
|
|
|
|
+ "15 blocks to reach the threshold 0.9990 of total blocks 15. " +
|
|
|
|
+ "Safe mode will be turned off automatically.", status);
|
|
|
|
+ assertFalse("Mis-replicated block queues should not be initialized " +
|
|
|
|
+ "until threshold is crossed",
|
|
|
|
+ NameNodeAdapter.safeModeInitializedReplQueues(nn));
|
|
|
|
+
|
|
|
|
+ cluster.restartDataNode(dnprops.remove(0));
|
|
|
|
+
|
|
|
|
+ // Wait for the block report from the restarted DN to come in.
|
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Boolean get() {
|
|
|
|
+ return NameNodeAdapter.getSafeModeSafeBlocks(nn) > 0;
|
|
|
|
+ }
|
|
|
|
+ }, 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());
|
|
|
|
+
|
|
|
|
+ assertTrue(NameNodeAdapter.safeModeInitializedReplQueues(nn));
|
|
|
|
+ assertEquals(15 - safe, nn.getNamesystem().getUnderReplicatedBlocks());
|
|
|
|
+
|
|
|
|
+ cluster.restartDataNodes();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test that, when under-replicated blocks are processed at the end of
|
|
|
|
+ * safe-mode, blocks currently under construction are not considered
|
|
|
|
+ * under-construction or missing. Regression test for HDFS-2822.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testRbwBlocksNotConsideredUnderReplicated() throws IOException {
|
|
|
|
+ List<FSDataOutputStream> stms = Lists.newArrayList();
|
|
|
|
+ try {
|
|
|
|
+ // Create some junk blocks so that the NN doesn't just immediately
|
|
|
|
+ // exit safemode on restart.
|
|
|
|
+ DFSTestUtil.createFile(fs, new Path("/junk-blocks"),
|
|
|
|
+ BLOCK_SIZE*4, (short)1, 1L);
|
|
|
|
+ // Create several files which are left open. It's important to
|
|
|
|
+ // create several here, because otherwise the first iteration of the
|
|
|
|
+ // replication monitor will pull them off the replication queue and
|
|
|
|
+ // hide this bug from the test!
|
|
|
|
+ for (int i = 0; i < 10; i++) {
|
|
|
|
+ FSDataOutputStream stm = fs.create(
|
|
|
|
+ new Path("/append-" + i), true, BLOCK_SIZE, (short) 1, BLOCK_SIZE);
|
|
|
|
+ stms.add(stm);
|
|
|
|
+ stm.write(1);
|
|
|
|
+ stm.hflush();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ cluster.restartNameNode();
|
|
|
|
+ FSNamesystem ns = cluster.getNameNode(0).getNamesystem();
|
|
|
|
+ BlockManagerTestUtil.updateState(ns.getBlockManager());
|
|
|
|
+ assertEquals(0, ns.getPendingReplicationBlocks());
|
|
|
|
+ assertEquals(0, ns.getCorruptReplicaBlocks());
|
|
|
|
+ assertEquals(0, ns.getMissingBlocksCount());
|
|
|
|
+
|
|
|
|
+ } finally {
|
|
|
|
+ for (FSDataOutputStream stm : stms) {
|
|
|
|
+ IOUtils.closeStream(stm);
|
|
|
|
+ }
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
public interface FSRun {
|
|
public interface FSRun {
|
|
public abstract void run(FileSystem fs) throws IOException;
|
|
public abstract void run(FileSystem fs) throws IOException;
|
|
@@ -241,4 +357,4 @@ public class TestSafeMode {
|
|
assertEquals("", cluster.getNamesystem().getSafemode());
|
|
assertEquals("", cluster.getNamesystem().getSafemode());
|
|
}
|
|
}
|
|
|
|
|
|
-}
|
|
|
|
|
|
+}
|