HDFS-2764. TestBackupNode is racy. Contributed by Aaron T. Myers.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1241780 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2012-02-08 05:38:29 +00:00
parent 11db1b855f
commit 4d3dc530e8
3 changed files with 24 additions and 14 deletions

View File

@ -197,6 +197,8 @@ Trunk (unreleased changes)
HDFS-2759. Pre-allocate HDFS edit log files after writing version number.
(atm)
HDFS-2764. TestBackupNode is racy. (atm)
Release 0.23.1 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -34,6 +34,7 @@
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@ -62,6 +63,8 @@
*/
public abstract class FSImageTestUtil {
public static final Log LOG = LogFactory.getLog(FSImageTestUtil.class);
/**
* The position in the fsimage header where the txid is
* written.
@ -379,6 +382,8 @@ static void assertNNHasCheckpoints(MiniDFSCluster cluster,
List<Integer> txids) {
for (File nameDir : getNameNodeCurrentDirs(cluster)) {
LOG.info("examining name dir with files: " +
Joiner.on(",").join(nameDir.listFiles()));
// Should have fsimage_N for the three checkpoints
for (long checkpointTxId : txids) {
File image = new File(nameDir,

View File

@ -17,14 +17,14 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.junit.Assert.*;
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collections;
import java.util.List;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
@ -41,13 +41,15 @@
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.junit.Before;
import org.junit.Test;
import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
public class TestBackupNode extends TestCase {
public class TestBackupNode {
public static final Log LOG = LogFactory.getLog(TestBackupNode.class);
@ -58,8 +60,8 @@ public class TestBackupNode extends TestCase {
static final String BASE_DIR = MiniDFSCluster.getBaseDirectory();
protected void setUp() throws Exception {
super.setUp();
@Before
public void setUp() throws Exception {
File baseDir = new File(BASE_DIR);
if(baseDir.exists())
if(!(FileUtil.fullyDelete(baseDir)))
@ -90,8 +92,7 @@ BackupNode startBackupNode(Configuration conf,
return (BackupNode)NameNode.createNameNode(new String[]{startupOpt.getName()}, c);
}
void waitCheckpointDone(
MiniDFSCluster cluster, BackupNode backup, long txid) {
void waitCheckpointDone(MiniDFSCluster cluster, long txid) {
long thisCheckpointTxId;
do {
try {
@ -99,16 +100,16 @@ void waitCheckpointDone(
"checkpoint txid should increase above " + txid);
Thread.sleep(1000);
} catch (Exception e) {}
thisCheckpointTxId = backup.getFSImage().getStorage()
// The checkpoint is not done until the nn has received it from the bn
thisCheckpointTxId = cluster.getNameNode().getFSImage().getStorage()
.getMostRecentCheckpointTxId();
} while (thisCheckpointTxId < txid);
// Check that the checkpoint got uploaded to NN successfully
FSImageTestUtil.assertNNHasCheckpoints(cluster,
Collections.singletonList((int)thisCheckpointTxId));
}
@Test
public void testCheckpointNode() throws Exception {
testCheckpoint(StartupOption.CHECKPOINT);
}
@ -118,6 +119,7 @@ public void testCheckpointNode() throws Exception {
* and keep in sync, even while the NN rolls, checkpoints
* occur, etc.
*/
@Test
public void testBackupNodeTailsEdits() throws Exception {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = null;
@ -235,6 +237,7 @@ private void assertStorageDirsMatch(final NameNode nn, final BackupNode backup)
FSImageTestUtil.assertParallelFilesAreIdentical(dirs, ImmutableSet.of("VERSION"));
}
@Test
public void testBackupNode() throws Exception {
testCheckpoint(StartupOption.BACKUP);
}
@ -274,7 +277,7 @@ void testCheckpoint(StartupOption op) throws Exception {
//
long txid = cluster.getNameNodeRpc().getTransactionID();
backup = startBackupNode(conf, op, 1);
waitCheckpointDone(cluster, backup, txid);
waitCheckpointDone(cluster, txid);
} catch(IOException e) {
LOG.error("Error in TestBackupNode:", e);
assertTrue(e.getLocalizedMessage(), false);
@ -309,7 +312,7 @@ void testCheckpoint(StartupOption op) throws Exception {
//
backup = startBackupNode(conf, op, 1);
long txid = cluster.getNameNodeRpc().getTransactionID();
waitCheckpointDone(cluster, backup, txid);
waitCheckpointDone(cluster, txid);
for (int i = 0; i < 10; i++) {
fileSys.mkdirs(new Path("file_" + i));
@ -317,11 +320,11 @@ void testCheckpoint(StartupOption op) throws Exception {
txid = cluster.getNameNodeRpc().getTransactionID();
backup.doCheckpoint();
waitCheckpointDone(cluster, backup, txid);
waitCheckpointDone(cluster, txid);
txid = cluster.getNameNodeRpc().getTransactionID();
backup.doCheckpoint();
waitCheckpointDone(cluster, backup, txid);
waitCheckpointDone(cluster, txid);
// Try BackupNode operations
InetSocketAddress add = backup.getNameNodeAddress();