|
@@ -18,14 +18,18 @@
|
|
|
package org.apache.hadoop.hdfs.qjournal.client;
|
|
|
|
|
|
import static org.junit.Assert.*;
|
|
|
+import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.JID;
|
|
|
+import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.FAKE_NSINFO;
|
|
|
+import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.writeSegment;
|
|
|
+import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.writeTxns;
|
|
|
+import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.verifyEdits;
|
|
|
|
|
|
import java.io.Closeable;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.net.InetSocketAddress;
|
|
|
import java.net.URISyntaxException;
|
|
|
-import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
-import java.util.SortedSet;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -33,14 +37,9 @@ import org.apache.commons.logging.impl.Log4JLogger;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
|
|
|
-import org.apache.hadoop.hdfs.qjournal.client.AsyncLogger;
|
|
|
-import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
|
|
|
-import org.apache.hadoop.hdfs.qjournal.client.QuorumException;
|
|
|
-import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
|
|
|
+import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
|
|
@@ -55,7 +54,6 @@ import org.junit.Test;
|
|
|
import org.mockito.Mockito;
|
|
|
|
|
|
import com.google.common.collect.Lists;
|
|
|
-import com.google.common.collect.Sets;
|
|
|
|
|
|
/**
|
|
|
* Functional tests for QuorumJournalManager.
|
|
@@ -65,9 +63,6 @@ public class TestQuorumJournalManager {
|
|
|
private static final Log LOG = LogFactory.getLog(
|
|
|
TestQuorumJournalManager.class);
|
|
|
|
|
|
- private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(
|
|
|
- 12345, "mycluster", "my-bp", 0L, 0);
|
|
|
- private static final String JID = "testQuorumJournalManager";
|
|
|
private MiniJournalCluster cluster;
|
|
|
private Configuration conf;
|
|
|
private QuorumJournalManager qjm;
|
|
@@ -95,18 +90,20 @@ public class TestQuorumJournalManager {
|
|
|
|
|
|
@After
|
|
|
public void shutdown() throws IOException {
|
|
|
- cluster.shutdown();
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testSingleWriter() throws Exception {
|
|
|
- writeSegment(qjm, 1, 3, true);
|
|
|
+ writeSegment(cluster, qjm, 1, 3, true);
|
|
|
|
|
|
// Should be finalized
|
|
|
checkRecovery(cluster, 1, 3);
|
|
|
|
|
|
// Start a new segment
|
|
|
- writeSegment(qjm, 4, 1, true);
|
|
|
+ writeSegment(cluster, qjm, 4, 1, true);
|
|
|
|
|
|
// Should be finalized
|
|
|
checkRecovery(cluster, 4, 4);
|
|
@@ -119,7 +116,7 @@ public class TestQuorumJournalManager {
|
|
|
List<EditLogInputStream> streams = Lists.newArrayList();
|
|
|
readerQjm.selectInputStreams(streams, 0, false);
|
|
|
assertEquals(0, streams.size());
|
|
|
- writeSegment(qjm, 1, 3, true);
|
|
|
+ writeSegment(cluster, qjm, 1, 3, true);
|
|
|
|
|
|
readerQjm.selectInputStreams(streams, 0, false);
|
|
|
try {
|
|
@@ -138,7 +135,7 @@ public class TestQuorumJournalManager {
|
|
|
|
|
|
// Ensure correct results when there is a stream in-progress, but we don't
|
|
|
// ask for in-progress.
|
|
|
- writeSegment(qjm, 4, 3, false);
|
|
|
+ writeSegment(cluster, qjm, 4, 3, false);
|
|
|
readerQjm.selectInputStreams(streams, 0, false);
|
|
|
try {
|
|
|
assertEquals(1, streams.size());
|
|
@@ -178,13 +175,13 @@ public class TestQuorumJournalManager {
|
|
|
*/
|
|
|
@Test
|
|
|
public void testOneJNMissingSegments() throws Exception {
|
|
|
- writeSegment(qjm, 1, 3, true);
|
|
|
+ writeSegment(cluster, qjm, 1, 3, true);
|
|
|
waitForAllPendingCalls(qjm.getLoggerSetForTests());
|
|
|
cluster.getJournalNode(0).stopAndJoin(0);
|
|
|
- writeSegment(qjm, 4, 3, true);
|
|
|
+ writeSegment(cluster, qjm, 4, 3, true);
|
|
|
waitForAllPendingCalls(qjm.getLoggerSetForTests());
|
|
|
cluster.restartJournalNode(0);
|
|
|
- writeSegment(qjm, 7, 3, true);
|
|
|
+ writeSegment(cluster, qjm, 7, 3, true);
|
|
|
waitForAllPendingCalls(qjm.getLoggerSetForTests());
|
|
|
cluster.getJournalNode(1).stopAndJoin(0);
|
|
|
|
|
@@ -199,37 +196,6 @@ public class TestQuorumJournalManager {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * TODO: this test needs to be fleshed out to be an exhaustive failure test
|
|
|
- * @throws Exception
|
|
|
- */
|
|
|
- @Test
|
|
|
- public void testOrchestratedFailures() throws Exception {
|
|
|
- writeSegment(qjm, 1, 3, true);
|
|
|
- writeSegment(qjm, 4, 3, true);
|
|
|
-
|
|
|
- SortedSet<Long> serials = Sets.newTreeSet();
|
|
|
- for (AsyncLogger l : qjm.getLoggerSetForTests().getLoggersForTests()) {
|
|
|
- IPCLoggerChannel ch = (IPCLoggerChannel)l;
|
|
|
- ch.waitForAllPendingCalls();
|
|
|
- serials.add(ch.getNextIpcSerial());
|
|
|
- }
|
|
|
-
|
|
|
- // All of the loggers should have sent the same number of RPCs, since there
|
|
|
- // were no failures.
|
|
|
- assertEquals(1, serials.size());
|
|
|
-
|
|
|
- long maxSerial = serials.first();
|
|
|
- LOG.info("Max IPC serial = " + maxSerial);
|
|
|
-
|
|
|
- cluster.shutdown();
|
|
|
-
|
|
|
- cluster = new MiniJournalCluster.Builder(conf)
|
|
|
- .build();
|
|
|
- qjm = createSpyingQJM();
|
|
|
- spies = qjm.getLoggerSetForTests().getLoggersForTests();
|
|
|
-
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Test case where a new writer picks up from an old one with no failures
|
|
@@ -238,8 +204,8 @@ public class TestQuorumJournalManager {
|
|
|
*/
|
|
|
@Test
|
|
|
public void testChangeWritersLogsInSync() throws Exception {
|
|
|
- writeSegment(qjm, 1, 3, false);
|
|
|
- assertExistsInQuorum(cluster,
|
|
|
+ writeSegment(cluster, qjm, 1, 3, false);
|
|
|
+ QJMTestUtil.assertExistsInQuorum(cluster,
|
|
|
NNStorage.getInProgressEditsFileName(1));
|
|
|
|
|
|
// Make a new QJM
|
|
@@ -301,7 +267,7 @@ public class TestQuorumJournalManager {
|
|
|
qe);
|
|
|
}
|
|
|
|
|
|
- assertExistsInQuorum(cluster,
|
|
|
+ QJMTestUtil.assertExistsInQuorum(cluster,
|
|
|
NNStorage.getInProgressEditsFileName(1));
|
|
|
|
|
|
// Shut down the specified JN, so it's not present during recovery.
|
|
@@ -320,7 +286,7 @@ public class TestQuorumJournalManager {
|
|
|
.when(spy).sendEdits(
|
|
|
Mockito.eq(txid), Mockito.eq(1), Mockito.<byte[]>any());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* edit lengths [3,4,5]
|
|
|
* first recovery:
|
|
@@ -389,7 +355,7 @@ public class TestQuorumJournalManager {
|
|
|
@Test
|
|
|
public void testPurgeLogs() throws Exception {
|
|
|
for (int txid = 1; txid <= 5; txid++) {
|
|
|
- writeSegment(qjm, txid, 1, true);
|
|
|
+ writeSegment(cluster, qjm, txid, 1, true);
|
|
|
}
|
|
|
File curDir = cluster.getCurrentDir(0, JID);
|
|
|
GenericTestUtils.assertGlobEquals(curDir, "edits_.*",
|
|
@@ -428,78 +394,18 @@ public class TestQuorumJournalManager {
|
|
|
|
|
|
private QuorumJournalManager createSpyingQJM()
|
|
|
throws IOException, URISyntaxException {
|
|
|
- return new QuorumJournalManager(
|
|
|
- conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO) {
|
|
|
- @Override
|
|
|
- protected List<AsyncLogger> createLoggers() throws IOException {
|
|
|
- List<AsyncLogger> realLoggers = super.createLoggers();
|
|
|
- List<AsyncLogger> spies = Lists.newArrayList();
|
|
|
- for (AsyncLogger logger : realLoggers) {
|
|
|
- spies.add(Mockito.spy(logger));
|
|
|
- }
|
|
|
- return spies;
|
|
|
- }
|
|
|
- };
|
|
|
- }
|
|
|
-
|
|
|
- private void writeSegment(QuorumJournalManager qjm,
|
|
|
- int startTxId, int numTxns, boolean finalize) throws IOException {
|
|
|
- EditLogOutputStream stm = qjm.startLogSegment(startTxId);
|
|
|
- // Should create in-progress
|
|
|
- assertExistsInQuorum(cluster,
|
|
|
- NNStorage.getInProgressEditsFileName(startTxId));
|
|
|
-
|
|
|
- writeTxns(stm, startTxId, numTxns);
|
|
|
- if (finalize) {
|
|
|
- stm.close();
|
|
|
- qjm.finalizeLogSegment(startTxId, startTxId + numTxns - 1);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void writeTxns(EditLogOutputStream stm, int startTxId, int numTxns)
|
|
|
- throws IOException {
|
|
|
- for (long txid = startTxId; txid < startTxId + numTxns; txid++) {
|
|
|
- TestQuorumJournalManagerUnit.writeOp(stm, txid);
|
|
|
- }
|
|
|
- stm.setReadyToFlush();
|
|
|
- stm.flush();
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Verify that the given list of streams contains exactly the range of
|
|
|
- * transactions specified, inclusive.
|
|
|
- */
|
|
|
- private void verifyEdits(List<EditLogInputStream> streams,
|
|
|
- int firstTxnId, int lastTxnId) throws IOException {
|
|
|
-
|
|
|
- Iterator<EditLogInputStream> iter = streams.iterator();
|
|
|
- assertTrue(iter.hasNext());
|
|
|
- EditLogInputStream stream = iter.next();
|
|
|
-
|
|
|
- for (int expected = firstTxnId;
|
|
|
- expected <= lastTxnId;
|
|
|
- expected++) {
|
|
|
-
|
|
|
- FSEditLogOp op = stream.readOp();
|
|
|
- while (op == null) {
|
|
|
- assertTrue("Expected to find txid " + expected + ", " +
|
|
|
- "but no more streams available to read from",
|
|
|
- iter.hasNext());
|
|
|
- stream = iter.next();
|
|
|
- op = stream.readOp();
|
|
|
+ AsyncLogger.Factory spyFactory = new AsyncLogger.Factory() {
|
|
|
+ @Override
|
|
|
+ public AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo,
|
|
|
+ String journalId, InetSocketAddress addr) {
|
|
|
+ return Mockito.spy(IPCLoggerChannel.FACTORY.createLogger(
|
|
|
+ conf, nsInfo, journalId, addr));
|
|
|
}
|
|
|
-
|
|
|
- assertEquals(FSEditLogOpCodes.OP_MKDIR, op.opCode);
|
|
|
- assertEquals(expected, op.getTransactionId());
|
|
|
- }
|
|
|
-
|
|
|
- assertNull(stream.readOp());
|
|
|
- assertFalse("Expected no more txns after " + lastTxnId +
|
|
|
- " but more streams are available", iter.hasNext());
|
|
|
+ };
|
|
|
+ return new QuorumJournalManager(
|
|
|
+ conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO, spyFactory);
|
|
|
}
|
|
|
|
|
|
-
|
|
|
-
|
|
|
private static void waitForAllPendingCalls(AsyncLoggerSet als)
|
|
|
throws InterruptedException {
|
|
|
for (AsyncLogger l : als.getLoggersForTests()) {
|
|
@@ -508,19 +414,6 @@ public class TestQuorumJournalManager {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void assertExistsInQuorum(MiniJournalCluster cluster,
|
|
|
- String fname) {
|
|
|
- int count = 0;
|
|
|
- for (int i = 0; i < 3; i++) {
|
|
|
- File dir = cluster.getCurrentDir(i, JID);
|
|
|
- if (new File(dir, fname).exists()) {
|
|
|
- count++;
|
|
|
- }
|
|
|
- }
|
|
|
- assertTrue("File " + fname + " should exist in a quorum of dirs",
|
|
|
- count >= cluster.getQuorumSize());
|
|
|
- }
|
|
|
-
|
|
|
private void checkRecovery(MiniJournalCluster cluster,
|
|
|
long segmentTxId, long expectedEndTxId)
|
|
|
throws IOException {
|