|
@@ -30,6 +30,8 @@ import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.URISyntaxException;
|
|
|
import java.util.List;
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
+import java.util.concurrent.Executors;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -54,6 +56,7 @@ import org.junit.Test;
|
|
|
import org.mockito.Mockito;
|
|
|
|
|
|
import com.google.common.collect.Lists;
|
|
|
+import com.google.common.util.concurrent.MoreExecutors;
|
|
|
|
|
|
/**
|
|
|
* Functional tests for QuorumJournalManager.
|
|
@@ -206,6 +209,124 @@ public class TestQuorumJournalManager {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test the case where the NN crashes after starting a new segment
|
|
|
+ * on all nodes, but before writing the first transaction to it.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testCrashAtBeginningOfSegment() throws Exception {
|
|
|
+ writeSegment(cluster, qjm, 1, 3, true);
|
|
|
+ waitForAllPendingCalls(qjm.getLoggerSetForTests());
|
|
|
+
|
|
|
+ EditLogOutputStream stm = qjm.startLogSegment(4);
|
|
|
+ try {
|
|
|
+ waitForAllPendingCalls(qjm.getLoggerSetForTests());
|
|
|
+ } finally {
|
|
|
+ stm.abort();
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ // Make a new QJM
|
|
|
+ qjm = new QuorumJournalManager(
|
|
|
+ conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO);
|
|
|
+ qjm.recoverUnfinalizedSegments();
|
|
|
+ checkRecovery(cluster, 1, 3);
|
|
|
+
|
|
|
+ writeSegment(cluster, qjm, 4, 3, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testOutOfSyncAtBeginningOfSegment0() throws Exception {
|
|
|
+ doTestOutOfSyncAtBeginningOfSegment(0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testOutOfSyncAtBeginningOfSegment1() throws Exception {
|
|
|
+ doTestOutOfSyncAtBeginningOfSegment(1);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testOutOfSyncAtBeginningOfSegment2() throws Exception {
|
|
|
+ doTestOutOfSyncAtBeginningOfSegment(2);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test the case where, at the beginning of a segment, transactions
|
|
|
+ * have been written to one JN but not others.
|
|
|
+ */
|
|
|
+ public void doTestOutOfSyncAtBeginningOfSegment(int nodeWithOneTxn)
|
|
|
+ throws Exception {
|
|
|
+
|
|
|
+ int nodeWithEmptySegment = (nodeWithOneTxn + 1) % 3;
|
|
|
+ int nodeMissingSegment = (nodeWithOneTxn + 2) % 3;
|
|
|
+
|
|
|
+ writeSegment(cluster, qjm, 1, 3, true);
|
|
|
+ waitForAllPendingCalls(qjm.getLoggerSetForTests());
|
|
|
+ cluster.getJournalNode(nodeMissingSegment).stopAndJoin(0);
|
|
|
+
|
|
|
+ // Open segment on 2/3 nodes
|
|
|
+ EditLogOutputStream stm = qjm.startLogSegment(4);
|
|
|
+ try {
|
|
|
+ waitForAllPendingCalls(qjm.getLoggerSetForTests());
|
|
|
+
|
|
|
+ // Write transactions to only 1/3 nodes
|
|
|
+ failLoggerAtTxn(spies.get(nodeWithEmptySegment), 4);
|
|
|
+ try {
|
|
|
+ writeTxns(stm, 4, 1);
|
|
|
+ fail("Did not fail even though 2/3 failed");
|
|
|
+ } catch (QuorumException qe) {
|
|
|
+ GenericTestUtils.assertExceptionContains("mock failure", qe);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ stm.abort();
|
|
|
+ }
|
|
|
+
|
|
|
+ // Bring back the down JN.
|
|
|
+ cluster.restartJournalNode(nodeMissingSegment);
|
|
|
+
|
|
|
+ // Make a new QJM. At this point, the state is as follows:
|
|
|
+ // A: nodeWithEmptySegment: 1-3 finalized, 4_inprogress (empty)
|
|
|
+ // B: nodeWithOneTxn: 1-3 finalized, 4_inprogress (1 txn)
|
|
|
+ // C: nodeMissingSegment: 1-3 finalized
|
|
|
+ GenericTestUtils.assertGlobEquals(
|
|
|
+ cluster.getCurrentDir(nodeWithEmptySegment, JID),
|
|
|
+ "edits_.*",
|
|
|
+ NNStorage.getFinalizedEditsFileName(1, 3),
|
|
|
+ NNStorage.getInProgressEditsFileName(4));
|
|
|
+ GenericTestUtils.assertGlobEquals(
|
|
|
+ cluster.getCurrentDir(nodeWithOneTxn, JID),
|
|
|
+ "edits_.*",
|
|
|
+ NNStorage.getFinalizedEditsFileName(1, 3),
|
|
|
+ NNStorage.getInProgressEditsFileName(4));
|
|
|
+ GenericTestUtils.assertGlobEquals(
|
|
|
+ cluster.getCurrentDir(nodeMissingSegment, JID),
|
|
|
+ "edits_.*",
|
|
|
+ NNStorage.getFinalizedEditsFileName(1, 3));
|
|
|
+
|
|
|
+
|
|
|
+ // Stop one of the nodes. Since we run this test three
|
|
|
+ // times, rotating the roles of the nodes, we'll test
|
|
|
+ // all the permutations.
|
|
|
+ cluster.getJournalNode(2).stopAndJoin(0);
|
|
|
+
|
|
|
+ qjm = createSpyingQJM();
|
|
|
+ qjm.recoverUnfinalizedSegments();
|
|
|
+
|
|
|
+ if (nodeWithOneTxn == 0 ||
|
|
|
+ nodeWithOneTxn == 1) {
|
|
|
+ // If the node that had the transaction committed was one of the nodes
|
|
|
+ // that responded during recovery, then we should have recovered txid
|
|
|
+ // 4.
|
|
|
+ checkRecovery(cluster, 4, 4);
|
|
|
+ writeSegment(cluster, qjm, 5, 3, true);
|
|
|
+ } else {
|
|
|
+ // Otherwise, we should have recovered only 1-3 and should be able to
|
|
|
+ // start a segment at 4.
|
|
|
+ checkRecovery(cluster, 1, 3);
|
|
|
+ writeSegment(cluster, qjm, 4, 3, true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
|
|
|
/**
|
|
|
* Test case where a new writer picks up from an old one with no failures
|
|
@@ -408,8 +529,15 @@ public class TestQuorumJournalManager {
|
|
|
@Override
|
|
|
public AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo,
|
|
|
String journalId, InetSocketAddress addr) {
|
|
|
- return Mockito.spy(IPCLoggerChannel.FACTORY.createLogger(
|
|
|
- conf, nsInfo, journalId, addr));
|
|
|
+ AsyncLogger logger = new IPCLoggerChannel(conf, nsInfo, journalId, addr) {
|
|
|
+ protected ExecutorService createExecutor() {
|
|
|
+ // Don't parallelize calls to the quorum in the tests.
|
|
|
+ // This makes the tests more deterministic.
|
|
|
+ return MoreExecutors.sameThreadExecutor();
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
+ return Mockito.spy(logger);
|
|
|
}
|
|
|
};
|
|
|
return new QuorumJournalManager(
|