|
@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerSafeMode.BMSafeModeStatus;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerSafeMode.BMSafeModeStatus;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
@@ -31,7 +32,6 @@ import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
|
|
-import org.mockito.Mockito;
|
|
|
|
import org.mockito.internal.util.reflection.Whitebox;
|
|
import org.mockito.internal.util.reflection.Whitebox;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
@@ -42,7 +42,6 @@ import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertFalse;
|
|
import static org.junit.Assert.assertFalse;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
-import static org.junit.Assert.fail;
|
|
|
|
import static org.mockito.Matchers.any;
|
|
import static org.mockito.Matchers.any;
|
|
import static org.mockito.Mockito.doReturn;
|
|
import static org.mockito.Mockito.doReturn;
|
|
import static org.mockito.Mockito.mock;
|
|
import static org.mockito.Mockito.mock;
|
|
@@ -92,9 +91,10 @@ public class TestBlockManagerSafeMode {
|
|
DATANODE_NUM);
|
|
DATANODE_NUM);
|
|
|
|
|
|
FSNamesystem fsn = mock(FSNamesystem.class);
|
|
FSNamesystem fsn = mock(FSNamesystem.class);
|
|
- Mockito.doReturn(true).when(fsn).hasWriteLock();
|
|
|
|
- Mockito.doReturn(true).when(fsn).hasReadLock();
|
|
|
|
- Mockito.doReturn(true).when(fsn).isRunning();
|
|
|
|
|
|
+ doReturn(true).when(fsn).hasWriteLock();
|
|
|
|
+ doReturn(true).when(fsn).hasReadLock();
|
|
|
|
+ doReturn(true).when(fsn).isRunning();
|
|
|
|
+ doReturn(true).when(fsn).isGenStampInFuture(any(Block.class));
|
|
NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
|
|
NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
|
|
|
|
|
|
bm = spy(new BlockManager(fsn, conf));
|
|
bm = spy(new BlockManager(fsn, conf));
|
|
@@ -110,7 +110,7 @@ public class TestBlockManagerSafeMode {
|
|
* Test set block total.
|
|
* Test set block total.
|
|
*
|
|
*
|
|
* The block total is set which will call checkSafeMode for the first time
|
|
* The block total is set which will call checkSafeMode for the first time
|
|
- * and bmSafeMode transfers from INITIALIZED to PENDING_THRESHOLD status
|
|
|
|
|
|
+ * and bmSafeMode transfers from OFF to PENDING_THRESHOLD status
|
|
*/
|
|
*/
|
|
@Test(timeout = 30000)
|
|
@Test(timeout = 30000)
|
|
public void testInitialize() {
|
|
public void testInitialize() {
|
|
@@ -170,7 +170,7 @@ public class TestBlockManagerSafeMode {
|
|
*
|
|
*
|
|
* Once the block threshold is reached, the block manger leaves safe mode and
|
|
* Once the block threshold is reached, the block manger leaves safe mode and
|
|
* increment will be a no-op.
|
|
* increment will be a no-op.
|
|
- * The safe mode status lifecycle: INITIALIZED -> PENDING_THRESHOLD -> OFF
|
|
|
|
|
|
+ * The safe mode status lifecycle: OFF -> PENDING_THRESHOLD -> OFF
|
|
*/
|
|
*/
|
|
@Test(timeout = 30000)
|
|
@Test(timeout = 30000)
|
|
public void testIncrementSafeBlockCount() {
|
|
public void testIncrementSafeBlockCount() {
|
|
@@ -198,7 +198,7 @@ public class TestBlockManagerSafeMode {
|
|
*
|
|
*
|
|
* Once the block threshold is reached, the block manger leaves safe mode and
|
|
* Once the block threshold is reached, the block manger leaves safe mode and
|
|
* increment will be a no-op.
|
|
* increment will be a no-op.
|
|
- * The safe mode status lifecycle: INITIALIZED -> PENDING_THRESHOLD -> EXTENSION-> OFF
|
|
|
|
|
|
+ * The safe mode status lifecycle: OFF -> PENDING_THRESHOLD -> EXTENSION-> OFF
|
|
*/
|
|
*/
|
|
@Test(timeout = 30000)
|
|
@Test(timeout = 30000)
|
|
public void testIncrementSafeBlockCountWithExtension() throws Exception {
|
|
public void testIncrementSafeBlockCountWithExtension() throws Exception {
|
|
@@ -220,7 +220,7 @@ public class TestBlockManagerSafeMode {
|
|
* Test that the block safe decreases the block safe.
|
|
* Test that the block safe decreases the block safe.
|
|
*
|
|
*
|
|
* The block manager stays in safe mode.
|
|
* The block manager stays in safe mode.
|
|
- * The safe mode status lifecycle: INITIALIZED -> PENDING_THRESHOLD
|
|
|
|
|
|
+ * The safe mode status lifecycle: OFF -> PENDING_THRESHOLD
|
|
*/
|
|
*/
|
|
@Test(timeout = 30000)
|
|
@Test(timeout = 30000)
|
|
public void testDecrementSafeBlockCount() {
|
|
public void testDecrementSafeBlockCount() {
|
|
@@ -242,7 +242,7 @@ public class TestBlockManagerSafeMode {
|
|
* Test when the block safe increment and decrement interleave.
|
|
* Test when the block safe increment and decrement interleave.
|
|
*
|
|
*
|
|
* Both the increment and decrement will be a no-op if the safe mode is OFF.
|
|
* Both the increment and decrement will be a no-op if the safe mode is OFF.
|
|
- * The safe mode status lifecycle: INITIALIZED -> PENDING_THRESHOLD -> OFF
|
|
|
|
|
|
+ * The safe mode status lifecycle: OFF -> PENDING_THRESHOLD -> OFF
|
|
*/
|
|
*/
|
|
@Test(timeout = 30000)
|
|
@Test(timeout = 30000)
|
|
public void testIncrementAndDecrementSafeBlockCount() {
|
|
public void testIncrementAndDecrementSafeBlockCount() {
|
|
@@ -309,24 +309,31 @@ public class TestBlockManagerSafeMode {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Test block manager won't leave safe mode if there are orphan blocks.
|
|
|
|
|
|
+ * Test block manager won't leave safe mode if there are blocks with
|
|
|
|
+ * generation stamp (GS) in future.
|
|
*/
|
|
*/
|
|
@Test(timeout = 30000)
|
|
@Test(timeout = 30000)
|
|
public void testStayInSafeModeWhenBytesInFuture() throws Exception {
|
|
public void testStayInSafeModeWhenBytesInFuture() throws Exception {
|
|
bmSafeMode.activate(BLOCK_TOTAL);
|
|
bmSafeMode.activate(BLOCK_TOTAL);
|
|
|
|
|
|
- when(bm.getBytesInFuture()).thenReturn(1L);
|
|
|
|
|
|
+ // Inject blocks with future GS
|
|
|
|
+ injectBlocksWithFugureGS(100L);
|
|
|
|
+ assertEquals(100L, bmSafeMode.getBytesInFuture());
|
|
|
|
+
|
|
// safe blocks are enough
|
|
// safe blocks are enough
|
|
setBlockSafe(BLOCK_THRESHOLD);
|
|
setBlockSafe(BLOCK_THRESHOLD);
|
|
|
|
|
|
// PENDING_THRESHOLD -> EXTENSION
|
|
// PENDING_THRESHOLD -> EXTENSION
|
|
bmSafeMode.checkSafeMode();
|
|
bmSafeMode.checkSafeMode();
|
|
- try {
|
|
|
|
- waitForExtensionPeriod();
|
|
|
|
- fail("Safe mode should not leave extension period with orphan blocks!");
|
|
|
|
- } catch (TimeoutException e) {
|
|
|
|
- assertEquals(BMSafeModeStatus.EXTENSION, getSafeModeStatus());
|
|
|
|
- }
|
|
|
|
|
|
+
|
|
|
|
+ assertFalse("Shouldn't leave safe mode in case of blocks with future GS! ",
|
|
|
|
+ bmSafeMode.leaveSafeMode(false));
|
|
|
|
+ assertTrue("Leaving safe mode forcefully should succeed regardless of " +
|
|
|
|
+ "blocks with future GS.", bmSafeMode.leaveSafeMode(true));
|
|
|
|
+ assertEquals("Number of blocks with future GS should have been cleared " +
|
|
|
|
+ "after leaving safe mode", 0L, bmSafeMode.getBytesInFuture());
|
|
|
|
+ assertTrue("Leaving safe mode should succeed after blocks with future GS " +
|
|
|
|
+ "are cleared.", bmSafeMode.leaveSafeMode(false));
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -353,7 +360,7 @@ public class TestBlockManagerSafeMode {
|
|
tip = bmSafeMode.getSafeModeTip();
|
|
tip = bmSafeMode.getSafeModeTip();
|
|
assertTrue(tip.contains(
|
|
assertTrue(tip.contains(
|
|
String.format("The reported blocks %d has reached the threshold"
|
|
String.format("The reported blocks %d has reached the threshold"
|
|
- + " %.4f of total blocks %d. ",
|
|
|
|
|
|
+ + " %.4f of total blocks %d. ",
|
|
getblockSafe(), THRESHOLD, BLOCK_TOTAL)));
|
|
getblockSafe(), THRESHOLD, BLOCK_TOTAL)));
|
|
assertTrue(tip.contains(
|
|
assertTrue(tip.contains(
|
|
String.format("The number of live datanodes %d has reached the " +
|
|
String.format("The number of live datanodes %d has reached the " +
|
|
@@ -363,7 +370,6 @@ public class TestBlockManagerSafeMode {
|
|
|
|
|
|
waitForExtensionPeriod();
|
|
waitForExtensionPeriod();
|
|
tip = bmSafeMode.getSafeModeTip();
|
|
tip = bmSafeMode.getSafeModeTip();
|
|
- System.out.println(tip);
|
|
|
|
assertTrue(tip.contains(
|
|
assertTrue(tip.contains(
|
|
String.format("The reported blocks %d has reached the threshold"
|
|
String.format("The reported blocks %d has reached the threshold"
|
|
+ " %.4f of total blocks %d. ",
|
|
+ " %.4f of total blocks %d. ",
|
|
@@ -375,7 +381,55 @@ public class TestBlockManagerSafeMode {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Mock block manager internal state for decrement safe block
|
|
|
|
|
|
+ * Test get safe mode tip in case of blocks with future GS.
|
|
|
|
+ */
|
|
|
|
+ @Test(timeout = 30000)
|
|
|
|
+ public void testGetSafeModeTipForBlocksWithFutureGS() throws Exception {
|
|
|
|
+ bmSafeMode.activate(BLOCK_TOTAL);
|
|
|
|
+
|
|
|
|
+ injectBlocksWithFugureGS(40L);
|
|
|
|
+ String tip = bmSafeMode.getSafeModeTip();
|
|
|
|
+ assertTrue(tip.contains(
|
|
|
|
+ String.format(
|
|
|
|
+ "The reported blocks %d needs additional %d blocks to reach the " +
|
|
|
|
+ "threshold %.4f of total blocks %d.%n",
|
|
|
|
+ 0, BLOCK_THRESHOLD, THRESHOLD, BLOCK_TOTAL)));
|
|
|
|
+ assertTrue(tip.contains(
|
|
|
|
+ "Name node detected blocks with generation stamps " +
|
|
|
|
+ "in future. This means that Name node metadata is inconsistent. " +
|
|
|
|
+ "This can happen if Name node metadata files have been manually " +
|
|
|
|
+ "replaced. Exiting safe mode will cause loss of " +
|
|
|
|
+ 40 + " byte(s). Please restart name node with " +
|
|
|
|
+ "right metadata or use \"hdfs dfsadmin -safemode forceExit\" " +
|
|
|
|
+ "if you are certain that the NameNode was started with the " +
|
|
|
|
+ "correct FsImage and edit logs. If you encountered this during " +
|
|
|
|
+ "a rollback, it is safe to exit with -safemode forceExit."
|
|
|
|
+ ));
|
|
|
|
+ assertFalse(tip.contains("Safe mode will be turned off"));
|
|
|
|
+
|
|
|
|
+ // blocks with future GS were already injected before.
|
|
|
|
+ setBlockSafe(BLOCK_THRESHOLD);
|
|
|
|
+ tip = bmSafeMode.getSafeModeTip();
|
|
|
|
+ assertTrue(tip.contains(
|
|
|
|
+ String.format("The reported blocks %d has reached the threshold"
|
|
|
|
+ + " %.4f of total blocks %d. ",
|
|
|
|
+ getblockSafe(), THRESHOLD, BLOCK_TOTAL)));
|
|
|
|
+ assertTrue(tip.contains(
|
|
|
|
+ "Name node detected blocks with generation stamps " +
|
|
|
|
+ "in future. This means that Name node metadata is inconsistent. " +
|
|
|
|
+ "This can happen if Name node metadata files have been manually " +
|
|
|
|
+ "replaced. Exiting safe mode will cause loss of " +
|
|
|
|
+ 40 + " byte(s). Please restart name node with " +
|
|
|
|
+ "right metadata or use \"hdfs dfsadmin -safemode forceExit\" " +
|
|
|
|
+ "if you are certain that the NameNode was started with the " +
|
|
|
|
+ "correct FsImage and edit logs. If you encountered this during " +
|
|
|
|
+ "a rollback, it is safe to exit with -safemode forceExit."
|
|
|
|
+ ));
|
|
|
|
+ assertFalse(tip.contains("Safe mode will be turned off"));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Mock block manager internal state for decrement safe block.
|
|
*/
|
|
*/
|
|
private void mockBlockManagerForBlockSafeDecrement() {
|
|
private void mockBlockManagerForBlockSafeDecrement() {
|
|
BlockInfo storedBlock = mock(BlockInfo.class);
|
|
BlockInfo storedBlock = mock(BlockInfo.class);
|
|
@@ -402,6 +456,12 @@ public class TestBlockManagerSafeMode {
|
|
}, EXTENSION / 10, EXTENSION * 2);
|
|
}, EXTENSION / 10, EXTENSION * 2);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private void injectBlocksWithFugureGS(long numBytesInFuture) {
|
|
|
|
+ BlockReportReplica brr = mock(BlockReportReplica.class);
|
|
|
|
+ when(brr.getBytesOnDisk()).thenReturn(numBytesInFuture);
|
|
|
|
+ bmSafeMode.checkBlocksWithFutureGS(brr);
|
|
|
|
+ }
|
|
|
|
+
|
|
private void setSafeModeStatus(BMSafeModeStatus status) {
|
|
private void setSafeModeStatus(BMSafeModeStatus status) {
|
|
Whitebox.setInternalState(bmSafeMode, "status", status);
|
|
Whitebox.setInternalState(bmSafeMode, "status", status);
|
|
}
|
|
}
|