HDFS-2730. Refactor shared HA-related test code into HATestUtil class. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1229023 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2012-01-09 04:12:37 +00:00
parent 89937fe7c7
commit 736b1860ab
11 changed files with 198 additions and 148 deletions

View File

@ -87,3 +87,5 @@ HDFS-2720. Fix MiniDFSCluster HA support to work properly on Windows. (Uma Mahes
HDFS-2291. Allow the StandbyNode to make checkpoints in an HA setup. (todd)
HDFS-2709. Appropriately handle error conditions in EditLogTailer (atm via todd)
HDFS-2730. Refactor shared HA-related test code into HATestUtil class (todd)

View File

@ -18,14 +18,12 @@
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
import static org.junit.Assert.*;
import java.io.IOException;
import java.io.OutputStream;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import org.apache.hadoop.conf.Configuration;
@ -34,6 +32,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.After;
import org.junit.Before;
@ -46,7 +45,6 @@ public class TestDFSClientFailover {
private Configuration conf = new Configuration();
private MiniDFSCluster cluster;
private static final String LOGICAL_HOSTNAME = "ha-nn-uri-%d";
@Before
public void setUpCluster() throws IOException {
@ -83,7 +81,7 @@ public class TestDFSClientFailover {
out1.close();
out2.close();
FileSystem fs = configureFailoverFs(cluster, conf);
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
AppendTestUtil.check(fs, TEST_FILE, FILE_LENGTH_TO_VERIFY);
cluster.getNameNode(0).stop();
@ -92,7 +90,7 @@ public class TestDFSClientFailover {
// Check that it functions even if the URL becomes canonicalized
// to include a port number.
Path withPort = new Path("hdfs://" +
getLogicalHostname(cluster) + ":" +
HATestUtil.getLogicalHostname(cluster) + ":" +
NameNode.DEFAULT_PORT + "/" + TEST_FILE.toUri().getPath());
FileSystem fs2 = withPort.getFileSystem(fs.getConf());
assertTrue(fs2.exists(withPort));
@ -117,38 +115,4 @@ public class TestDFSClientFailover {
"does not use port information", ioe);
}
}
public static FileSystem configureFailoverFs(MiniDFSCluster cluster, Configuration conf)
throws IOException, URISyntaxException {
InetSocketAddress nnAddr1 = cluster.getNameNode(0).getNameNodeAddress();
InetSocketAddress nnAddr2 = cluster.getNameNode(1).getNameNodeAddress();
String nsId = "nameserviceId1";
String nameNodeId1 = "nn1";
String nameNodeId2 = "nn2";
String logicalName = getLogicalHostname(cluster);
conf = new Configuration(conf);
String address1 = "hdfs://" + nnAddr1.getHostName() + ":" + nnAddr1.getPort();
String address2 = "hdfs://" + nnAddr2.getHostName() + ":" + nnAddr2.getPort();
conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
nsId, nameNodeId1), address1);
conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
nsId, nameNodeId2), address2);
conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, nsId);
conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY, nsId),
nameNodeId1 + "," + nameNodeId2);
conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
ConfiguredFailoverProxyProvider.class.getName());
FileSystem fs = FileSystem.get(new URI("hdfs://" + logicalName), conf);
return fs;
}
private static String getLogicalHostname(MiniDFSCluster cluster) {
return String.format(LOGICAL_HOSTNAME, cluster.getInstanceId());
}
}

View File

@ -0,0 +1,162 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode.ha;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.concurrent.TimeoutException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.test.GenericTestUtils;
import com.google.common.base.Supplier;
/**
* Static utility functions useful for testing HA.
*/
public abstract class HATestUtil {
private static Log LOG = LogFactory.getLog(HATestUtil.class);
private static final String LOGICAL_HOSTNAME = "ha-nn-uri-%d";
/**
* Trigger an edits log roll on the active and then wait for the standby to
* catch up to all the edits done by the active. This method will check
* repeatedly for up to NN_LAG_TIMEOUT milliseconds, and then fail throwing
* {@link CouldNotCatchUpException}
*
* @param active active NN
* @param standby standby NN which should catch up to active
* @throws IOException if an error occurs rolling the edit log
* @throws CouldNotCatchUpException if the standby doesn't catch up to the
* active in NN_LAG_TIMEOUT milliseconds
*/
static void waitForStandbyToCatchUp(NameNode active,
NameNode standby) throws InterruptedException, IOException, CouldNotCatchUpException {
long activeTxId = active.getNamesystem().getFSImage().getEditLog()
.getLastWrittenTxId();
active.getRpcServer().rollEditLog();
long start = System.currentTimeMillis();
while (System.currentTimeMillis() - start < TestEditLogTailer.NN_LAG_TIMEOUT) {
long nn2HighestTxId = standby.getNamesystem().getFSImage()
.getLastAppliedTxId();
if (nn2HighestTxId >= activeTxId) {
return;
}
Thread.sleep(TestEditLogTailer.SLEEP_TIME);
}
throw new CouldNotCatchUpException("Standby did not catch up to txid " +
activeTxId + " (currently at " +
standby.getNamesystem().getFSImage().getLastAppliedTxId() + ")");
}
/**
* Wait for the datanodes in the cluster to process any block
* deletions that have already been asynchronously queued.
*/
static void waitForDNDeletions(final MiniDFSCluster cluster)
throws TimeoutException, InterruptedException {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
for (DataNode dn : cluster.getDataNodes()) {
if (DataNodeAdapter.getPendingAsyncDeletions(dn) > 0) {
return false;
}
}
return true;
}
}, 1000, 10000);
}
/**
* Wait for the NameNode to issue any deletions that are already
* pending (i.e. for the pendingDeletionBlocksCount to go to 0)
*/
static void waitForNNToIssueDeletions(final NameNode nn)
throws Exception {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
LOG.info("Waiting for NN to issue block deletions to DNs");
return nn.getNamesystem().getBlockManager().getPendingDeletionBlocksCount() == 0;
}
}, 250, 10000);
}
public static class CouldNotCatchUpException extends IOException {
private static final long serialVersionUID = 1L;
public CouldNotCatchUpException(String message) {
super(message);
}
}
public static FileSystem configureFailoverFs(MiniDFSCluster cluster, Configuration conf)
throws IOException, URISyntaxException {
InetSocketAddress nnAddr1 = cluster.getNameNode(0).getNameNodeAddress();
InetSocketAddress nnAddr2 = cluster.getNameNode(1).getNameNodeAddress();
String nsId = "nameserviceId1";
String nameNodeId1 = "nn1";
String nameNodeId2 = "nn2";
String logicalName = getLogicalHostname(cluster);
conf = new Configuration(conf);
String address1 = "hdfs://" + nnAddr1.getHostName() + ":" + nnAddr1.getPort();
String address2 = "hdfs://" + nnAddr2.getHostName() + ":" + nnAddr2.getPort();
conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
nsId, nameNodeId1), address1);
conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
nsId, nameNodeId2), address2);
conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, nsId);
conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY, nsId),
nameNodeId1 + "," + nameNodeId2);
conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
ConfiguredFailoverProxyProvider.class.getName());
FileSystem fs = FileSystem.get(new URI("hdfs://" + logicalName), conf);
return fs;
}
public static String getLogicalHostname(MiniDFSCluster cluster) {
return String.format(LOGICAL_HOSTNAME, cluster.getInstanceId());
}
}

View File

@ -25,7 +25,6 @@ import java.io.StringWriter;
import java.net.URISyntaxException;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.TimeoutException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -39,7 +38,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.TestDFSClientFailover;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@ -48,7 +46,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@ -111,7 +108,7 @@ public class TestDNFencing {
cluster.triggerBlockReports();
nn2.getNamesystem().getEditLogTailer().setSleepTime(250);
nn2.getNamesystem().getEditLogTailer().interrupt();
fs = TestDFSClientFailover.configureFailoverFs(cluster, conf);
fs = HATestUtil.configureFailoverFs(cluster, conf);
}
@After
@ -172,7 +169,7 @@ public class TestDNFencing {
BlockManagerTestUtil.computeInvalidationWork(
nn2.getNamesystem().getBlockManager());
cluster.triggerHeartbeats();
waitForDNDeletions(cluster);
HATestUtil.waitForDNDeletions(cluster);
cluster.triggerDeletionReports();
assertEquals(0, nn2.getNamesystem().getUnderReplicatedBlocks());
assertEquals(0, nn2.getNamesystem().getPendingReplicationBlocks());
@ -258,9 +255,9 @@ public class TestDNFencing {
BlockManagerTestUtil.computeInvalidationWork(
nn2.getNamesystem().getBlockManager());
waitForNNToIssueDeletions(nn2);
HATestUtil.waitForNNToIssueDeletions(nn2);
cluster.triggerHeartbeats();
waitForDNDeletions(cluster);
HATestUtil.waitForDNDeletions(cluster);
cluster.triggerDeletionReports();
assertEquals(0, nn2.getNamesystem().getUnderReplicatedBlocks());
assertEquals(0, nn2.getNamesystem().getPendingReplicationBlocks());
@ -283,7 +280,7 @@ public class TestDNFencing {
DFSTestUtil.createFile(fs, TEST_FILE_PATH, 30*SMALL_BLOCK, (short)1, 1L);
banner("rolling NN1's edit log, forcing catch-up");
TestEditLogTailer.waitForStandbyToCatchUp(nn1, nn2);
HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
// Get some new replicas reported so that NN2 now considers
// them over-replicated and schedules some more deletions
@ -353,9 +350,9 @@ public class TestDNFencing {
BlockManagerTestUtil.computeInvalidationWork(
nn2.getNamesystem().getBlockManager());
waitForNNToIssueDeletions(nn2);
HATestUtil.waitForNNToIssueDeletions(nn2);
cluster.triggerHeartbeats();
waitForDNDeletions(cluster);
HATestUtil.waitForDNDeletions(cluster);
cluster.triggerDeletionReports();
assertEquals(0, nn2.getNamesystem().getUnderReplicatedBlocks());
assertEquals(0, nn2.getNamesystem().getPendingReplicationBlocks());
@ -410,33 +407,6 @@ public class TestDNFencing {
return count;
}
static void waitForDNDeletions(final MiniDFSCluster cluster)
throws TimeoutException, InterruptedException {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
for (DataNode dn : cluster.getDataNodes()) {
if (DataNodeAdapter.getPendingAsyncDeletions(dn) > 0) {
return false;
}
}
return true;
}
}, 1000, 10000);
}
static void waitForNNToIssueDeletions(final NameNode nn)
throws Exception {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
LOG.info("Waiting for NN to issue block deletions to DNs");
return nn.getNamesystem().getBlockManager().getPendingDeletionBlocksCount() == 0;
}
}, 250, 10000);
}
/**
* A BlockPlacementPolicy which, rather than using space available, makes
* random decisions about which excess replica to delete. This is because,

View File

@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.TestDFSClientFailover;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
@ -132,7 +131,7 @@ public class TestDNFencingWithReplication {
nn2.getNamesystem().getEditLogTailer().setSleepTime(250);
nn2.getNamesystem().getEditLogTailer().interrupt();
FileSystem fs = TestDFSClientFailover.configureFailoverFs(
FileSystem fs = HATestUtil.configureFailoverFs(
cluster, conf);
TestContext togglers = new TestContext();
for (int i = 0; i < NUM_THREADS; i++) {

View File

@ -34,15 +34,14 @@ import org.apache.hadoop.hdfs.server.namenode.FSImage;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.log4j.Level;
import org.junit.Assert;
import org.junit.Test;
public class TestEditLogTailer {
private static final String DIR_PREFIX = "/dir";
private static final int DIRS_TO_MAKE = 20;
private static final long SLEEP_TIME = 1000;
private static final long NN_LAG_TIMEOUT = 10 * 1000;
static final long SLEEP_TIME = 1000;
static final long NN_LAG_TIMEOUT = 10 * 1000;
static {
((Log4JLogger)FSImage.LOG).getLogger().setLevel(Level.ALL);
@ -74,7 +73,7 @@ public class TestEditLogTailer {
true);
}
waitForStandbyToCatchUp(nn1, nn2);
HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
for (int i = 0; i < DIRS_TO_MAKE / 2; i++) {
assertTrue(NameNodeAdapter.getFileInfo(nn2,
@ -87,7 +86,7 @@ public class TestEditLogTailer {
true);
}
waitForStandbyToCatchUp(nn1, nn2);
HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
assertTrue(NameNodeAdapter.getFileInfo(nn2,
@ -101,45 +100,4 @@ public class TestEditLogTailer {
private static String getDirPath(int suffix) {
return DIR_PREFIX + suffix;
}
/**
* Trigger an edits log roll on the active and then wait for the standby to
* catch up to all the edits done by the active. This method will check
* repeatedly for up to NN_LAG_TIMEOUT milliseconds, and then fail throwing
* {@link CouldNotCatchUpException}.
*
* @param active active NN
* @param standby standby NN which should catch up to active
* @throws IOException if an error occurs rolling the edit log
* @throws CouldNotCatchUpException if the standby doesn't catch up to the
* active in NN_LAG_TIMEOUT milliseconds
*/
static void waitForStandbyToCatchUp(NameNode active,
NameNode standby) throws InterruptedException, IOException, CouldNotCatchUpException {
long activeTxId = active.getNamesystem().getFSImage().getEditLog()
.getLastWrittenTxId();
active.getRpcServer().rollEditLog();
long start = System.currentTimeMillis();
while (System.currentTimeMillis() - start < NN_LAG_TIMEOUT) {
long nn2HighestTxId = standby.getNamesystem().getFSImage()
.getLastAppliedTxId();
if (nn2HighestTxId >= activeTxId) {
return;
}
Thread.sleep(SLEEP_TIME);
}
throw new CouldNotCatchUpException("Standby did not catch up to txid " +
activeTxId + " (currently at " +
standby.getNamesystem().getFSImage().getLastAppliedTxId() + ")");
}
public static class CouldNotCatchUpException extends IOException {
public CouldNotCatchUpException(String message) {
super(message);
}
}
}

View File

@ -41,13 +41,12 @@ import org.apache.hadoop.ha.ServiceFailedException;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.TestDFSClientFailover;
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.hdfs.server.namenode.ha.TestEditLogTailer.CouldNotCatchUpException;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil.CouldNotCatchUpException;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@ -84,9 +83,9 @@ public class TestFailureToReadEdits {
nn2.getNamesystem().getEditLogTailer().interrupt();
nn2.getNamesystem().getEditLogTailer().setRuntime(mockRuntime);
FileSystem fs = TestDFSClientFailover.configureFailoverFs(cluster, conf);
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
fs.mkdirs(new Path(TEST_DIR1));
TestEditLogTailer.waitForStandbyToCatchUp(nn1, nn2);
HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
// If these two ops are applied twice, the first op will throw an
// exception the second time its replayed.
@ -107,9 +106,9 @@ public class TestFailureToReadEdits {
nn2.getNamesystem().getEditLogTailer().setEditLog(spyEditLog);
try {
TestEditLogTailer.waitForStandbyToCatchUp(nn1, nn2);
HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
fail("Standby fully caught up, but should not have been able to");
} catch (CouldNotCatchUpException e) {
} catch (HATestUtil.CouldNotCatchUpException e) {
verify(mockRuntime, times(0)).exit(anyInt());
}
@ -125,7 +124,7 @@ public class TestFailureToReadEdits {
// Now let the standby read ALL the edits.
answer.setThrowExceptionOnRead(false);
TestEditLogTailer.waitForStandbyToCatchUp(nn1, nn2);
HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
// Null because it was deleted.
assertNull(NameNodeAdapter.getFileInfo(nn2,

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.TestDFSClientFailover;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@ -68,7 +67,7 @@ public class TestHASafeMode {
nn0 = cluster.getNameNode(0);
nn1 = cluster.getNameNode(1);
fs = TestDFSClientFailover.configureFailoverFs(cluster, conf);
fs = HATestUtil.configureFailoverFs(cluster, conf);
nn0.getNamesystem().getEditLogTailer().setRuntime(mockRuntime);
@ -126,7 +125,7 @@ public class TestHASafeMode {
"The reported blocks 0 needs additional 3 blocks to reach"));
banner("Waiting for standby to catch up to active namespace");
TestEditLogTailer.waitForStandbyToCatchUp(nn0, nn1);
HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
status = nn1.getNamesystem().getSafemode();
assertTrue("Bad safemode status: '" + status + "'",
@ -167,7 +166,7 @@ public class TestHASafeMode {
banner("Waiting for standby to catch up to active namespace");
TestEditLogTailer.waitForStandbyToCatchUp(nn0, nn1);
HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
status = nn1.getNamesystem().getSafemode();
assertTrue("Bad safemode status: '" + status + "'",
@ -221,7 +220,7 @@ public class TestHASafeMode {
"The reported blocks 0 needs additional 5 blocks to reach"));
banner("Waiting for standby to catch up to active namespace");
TestEditLogTailer.waitForStandbyToCatchUp(nn0, nn1);
HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
status = nn1.getNamesystem().getSafemode();
assertTrue("Bad safemode status: '" + status + "'",
status.startsWith(
@ -265,7 +264,7 @@ public class TestHASafeMode {
banner("Triggering deletions on DNs and Deletion Reports");
cluster.triggerHeartbeats();
TestDNFencing.waitForDNDeletions(cluster);
HATestUtil.waitForDNDeletions(cluster);
cluster.triggerDeletionReports();
status = nn1.getNamesystem().getSafemode();
@ -275,7 +274,7 @@ public class TestHASafeMode {
"The reported blocks 0 needs additional 10 blocks"));
banner("Waiting for standby to catch up to active namespace");
TestEditLogTailer.waitForStandbyToCatchUp(nn0, nn1);
HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
status = nn1.getNamesystem().getSafemode();
assertTrue("Bad safemode status: '" + status + "'",

View File

@ -29,7 +29,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.TestDFSClientFailover;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
@ -110,7 +109,7 @@ public class TestHAStateTransitions {
cluster.transitionToActive(0);
LOG.info("Starting with NN 0 active");
FileSystem fs = TestDFSClientFailover.configureFailoverFs(cluster, conf);
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
fs.mkdirs(TEST_DIR);
LOG.info("Failing over to NN 1");
@ -161,7 +160,7 @@ public class TestHAStateTransitions {
Mockito.doAnswer(new GenericTestUtils.SleepAnswer(50))
.when(spyLock).writeLock();
final FileSystem fs = TestDFSClientFailover.configureFailoverFs(
final FileSystem fs = HATestUtil.configureFailoverFs(
cluster, conf);
TestContext ctx = new TestContext();

View File

@ -30,7 +30,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.TestDFSClientFailover;
import org.apache.hadoop.hdfs.server.namenode.FSImage;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@ -72,7 +71,7 @@ public class TestStandbyCheckpoints {
nn0 = cluster.getNameNode(0);
nn1 = cluster.getNameNode(1);
fs = TestDFSClientFailover.configureFailoverFs(cluster, conf);
fs = HATestUtil.configureFailoverFs(cluster, conf);
nn1.getNamesystem().getEditLogTailer().setSleepTime(250);
nn1.getNamesystem().getEditLogTailer().interrupt();
@ -91,7 +90,7 @@ public class TestStandbyCheckpoints {
public void testSBNCheckpoints() throws Exception {
doEdits(0, 10);
TestEditLogTailer.waitForStandbyToCatchUp(nn0, nn1);
HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
// Once the standby catches up, it should notice that it needs to
// do a checkpoint and save one to its local directories.
waitForCheckpoint(1, ImmutableList.of(0, 12));
@ -162,7 +161,7 @@ public class TestStandbyCheckpoints {
.saveNamespace((FSNamesystem) Mockito.anyObject());
// Roll the primary and wait for the standby to catch up
TestEditLogTailer.waitForStandbyToCatchUp(nn0, nn1);
HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
Thread.sleep(2000);
// We should make exactly one checkpoint at this new txid.

View File

@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.TestDFSClientFailover;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@ -91,7 +90,7 @@ public class TestStandbyIsHot {
nn2.getNamesystem().getEditLogTailer().setSleepTime(250);
nn2.getNamesystem().getEditLogTailer().interrupt();
FileSystem fs = TestDFSClientFailover.configureFailoverFs(cluster, conf);
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
Thread.sleep(1000);
System.err.println("==================================");