HDFS-2804. Should not mark blocks under-replicated when exiting safemode. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1235033 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2012-01-23 22:22:46 +00:00
parent 8d1c996242
commit 465663afbf
3 changed files with 64 additions and 5 deletions

View File

@ -125,3 +125,5 @@ HDFS-2737. Automatically trigger log rolls periodically on the active NN. (todd
HDFS-2820. Add a simple sanity check for HA config (todd) HDFS-2820. Add a simple sanity check for HA config (todd)
HDFS-2688. Add tests for quota tracking in an HA cluster. (todd) HDFS-2688. Add tests for quota tracking in an HA cluster. (todd)
HDFS-2804. Should not mark blocks under-replicated when exiting safemode (todd)

View File

@ -848,6 +848,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return fsRunning; return fsRunning;
} }
private boolean isInStandbyState() {
if (haContext == null || haContext.getState() == null) {
// We're still starting up. In this case, if HA is
// on for the cluster, we always start in standby. Otherwise
// start in active.
return haEnabled;
}
return haContext.getState() instanceof StandbyState;
}
/** /**
* Dump all metadata into specified file * Dump all metadata into specified file
*/ */
@ -3345,8 +3356,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return; return;
} }
} }
// if not done yet, initialize replication queues // if not done yet, initialize replication queues.
if (!isPopulatingReplQueues()) { // In the standby, do not populate repl queues
if (!isPopulatingReplQueues() && !isInStandbyState()) {
initializeReplQueues(); initializeReplQueues();
} }
long timeInSafemode = now() - systemStart; long timeInSafemode = now() - systemStart;
@ -3389,7 +3401,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* initializing replication queues. * initializing replication queues.
*/ */
private synchronized boolean canInitializeReplQueues() { private synchronized boolean canInitializeReplQueues() {
return blockSafe >= blockReplQueueThreshold; return !isInStandbyState() && blockSafe >= blockReplQueueThreshold;
} }
/** /**
@ -3705,8 +3717,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
@Override @Override
public boolean isPopulatingReplQueues() { public boolean isPopulatingReplQueues() {
if (haContext != null && // null during startup! if (isInStandbyState()) {
!haContext.getState().shouldPopulateReplQueues()) {
return false; return false;
} }
// safeMode is volatile, and may be set to null at any time // safeMode is volatile, and may be set to null at any time

View File

@ -17,6 +17,7 @@
*/ */
package org.apache.hadoop.hdfs.server.namenode.ha; package org.apache.hadoop.hdfs.server.namenode.ha;
import static org.junit.Assert.*;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyInt;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
@ -32,16 +33,21 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import com.google.common.base.Supplier;
/** /**
* Tests that exercise safemode in an HA cluster. * Tests that exercise safemode in an HA cluster.
*/ */
@ -59,6 +65,8 @@ public class TestHASafeMode {
Configuration conf = new Configuration(); Configuration conf = new Configuration();
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
cluster = new MiniDFSCluster.Builder(conf) cluster = new MiniDFSCluster.Builder(conf)
.nnTopology(MiniDFSNNTopology.simpleHATopology()) .nnTopology(MiniDFSNNTopology.simpleHATopology())
.numDataNodes(3) .numDataNodes(3)
@ -425,6 +433,44 @@ public class TestHASafeMode {
"total blocks 6. Safe mode will be turned off automatically")); "total blocks 6. Safe mode will be turned off automatically"));
} }
/**
* Regression test for HDFS-2804: standby should not populate replication
* queues when exiting safe mode.
*/
@Test
public void testNoPopulatingReplQueuesWhenExitingSafemode() throws Exception {
DFSTestUtil.createFile(fs, new Path("/test"), 15*BLOCK_SIZE, (short)3, 1L);
HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
// get some blocks in the SBN's image
nn1.getRpcServer().setSafeMode(SafeModeAction.SAFEMODE_ENTER);
NameNodeAdapter.saveNamespace(nn1);
nn1.getRpcServer().setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
// and some blocks in the edit logs
DFSTestUtil.createFile(fs, new Path("/test2"), 15*BLOCK_SIZE, (short)3, 1L);
nn0.getRpcServer().rollEditLog();
cluster.stopDataNode(1);
cluster.shutdownNameNode(1);
//Configuration sbConf = cluster.getConfiguration(1);
//sbConf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 1);
cluster.restartNameNode(1, false);
nn1 = cluster.getNameNode(1);
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
return !nn1.isInSafeMode();
}
}, 100, 10000);
BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
assertEquals(0L, nn1.getNamesystem().getUnderReplicatedBlocks());
assertEquals(0L, nn1.getNamesystem().getPendingReplicationBlocks());
}
/** /**
* Print a big banner in the test log to make debug easier. * Print a big banner in the test log to make debug easier.
*/ */