|
@@ -17,10 +17,9 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.qjournal.client;
|
|
|
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
-import static org.junit.Assert.fail;
|
|
|
+import static org.junit.Assert.*;
|
|
|
|
|
|
+import java.io.Closeable;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.net.URISyntaxException;
|
|
@@ -35,11 +34,15 @@ 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.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;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
@@ -97,6 +100,69 @@ public class TestQuorumJournalManager {
|
|
|
checkRecovery(cluster, 4, 4);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testReaderWhileAnotherWrites() throws Exception {
|
|
|
+
|
|
|
+ QuorumJournalManager readerQjm = createSpyingQJM();
|
|
|
+ List<EditLogInputStream> streams = Lists.newArrayList();
|
|
|
+ readerQjm.selectInputStreams(streams, 0, false);
|
|
|
+ assertEquals(0, streams.size());
|
|
|
+ writeSegment(qjm, 1, 3, true);
|
|
|
+
|
|
|
+ readerQjm.selectInputStreams(streams, 0, false);
|
|
|
+ try {
|
|
|
+ assertEquals(1, streams.size());
|
|
|
+ // Validate the actual stream contents.
|
|
|
+ EditLogInputStream stream = streams.get(0);
|
|
|
+ assertEquals(1, stream.getFirstTxId());
|
|
|
+ assertEquals(3, stream.getLastTxId());
|
|
|
+
|
|
|
+ for (int i = 1; i <= 3; i++) {
|
|
|
+ FSEditLogOp op = stream.readOp();
|
|
|
+ assertEquals(FSEditLogOpCodes.OP_MKDIR, op.opCode);
|
|
|
+ assertEquals(i, op.getTransactionId());
|
|
|
+ }
|
|
|
+ assertNull(stream.readOp());
|
|
|
+ } finally {
|
|
|
+ IOUtils.cleanup(LOG, streams.toArray(new Closeable[0]));
|
|
|
+ streams.clear();
|
|
|
+ }
|
|
|
+
|
|
|
+ // Ensure correct results when there is a stream in-progress, but we don't
|
|
|
+ // ask for in-progress.
|
|
|
+ writeSegment(qjm, 4, 3, false);
|
|
|
+ readerQjm.selectInputStreams(streams, 0, false);
|
|
|
+ try {
|
|
|
+ assertEquals(1, streams.size());
|
|
|
+ EditLogInputStream stream = streams.get(0);
|
|
|
+ assertEquals(1, stream.getFirstTxId());
|
|
|
+ assertEquals(3, stream.getLastTxId());
|
|
|
+ } finally {
|
|
|
+ IOUtils.cleanup(LOG, streams.toArray(new Closeable[0]));
|
|
|
+ streams.clear();
|
|
|
+ }
|
|
|
+
|
|
|
+ // TODO: check results for selectInputStreams with inProgressOK = true.
|
|
|
+ // This doesn't currently work, due to a bug where RedundantEditInputStream
|
|
|
+ // throws an exception if there are any unvalidated in-progress edits in the list!
|
|
|
+ // But, it shouldn't be necessary for current use cases.
|
|
|
+
|
|
|
+ qjm.finalizeLogSegment(4, 6);
|
|
|
+ readerQjm.selectInputStreams(streams, 0, false);
|
|
|
+ try {
|
|
|
+ assertEquals(2, streams.size());
|
|
|
+ assertEquals(4, streams.get(1).getFirstTxId());
|
|
|
+ assertEquals(6, streams.get(1).getLastTxId());
|
|
|
+ } finally {
|
|
|
+ IOUtils.cleanup(LOG, streams.toArray(new Closeable[0]));
|
|
|
+ streams.clear();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * 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);
|
|
@@ -287,7 +353,6 @@ public class TestQuorumJournalManager {
|
|
|
conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO) {
|
|
|
@Override
|
|
|
protected List<AsyncLogger> createLoggers() throws IOException {
|
|
|
- LOG.info("===> make spies");
|
|
|
List<AsyncLogger> realLoggers = super.createLoggers();
|
|
|
List<AsyncLogger> spies = Lists.newArrayList();
|
|
|
for (AsyncLogger logger : realLoggers) {
|