|
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.JID;
|
|
|
import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.verifyEdits;
|
|
|
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.client.SpyQJournalUtil.spyGetJournaledEdits;
|
|
|
import static org.apache.hadoop.hdfs.qjournal.client.TestQuorumJournalManagerUnit.futureThrows;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
@@ -34,12 +35,10 @@ import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.URI;
|
|
|
-import java.net.URISyntaxException;
|
|
|
import java.net.URL;
|
|
|
import java.net.UnknownHostException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
-import java.util.concurrent.ExecutorService;
|
|
|
import java.util.concurrent.Semaphore;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
@@ -59,7 +58,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
|
|
|
import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
|
|
|
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
|
|
|
-import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
|
|
|
import org.apache.hadoop.hdfs.qjournal.server.JournalFaultInjector;
|
|
|
import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
|
@@ -68,7 +66,6 @@ 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.namenode.NameNodeLayoutVersion;
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.ipc.ProtobufRpcEngine2;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
@@ -1135,9 +1132,9 @@ public class TestQuorumJournalManager {
|
|
|
writeTxns(stm, 21, 20);
|
|
|
|
|
|
Semaphore semaphore = new Semaphore(0);
|
|
|
- spyGetJournaledEdits(0, 21, () -> semaphore.release(1));
|
|
|
- spyGetJournaledEdits(1, 21, () -> semaphore.release(1));
|
|
|
- spyGetJournaledEdits(2, 21, () -> semaphore.acquireUninterruptibly(2));
|
|
|
+ spyGetJournaledEdits(spies, 0, 21, () -> semaphore.release(1));
|
|
|
+ spyGetJournaledEdits(spies, 1, 21, () -> semaphore.release(1));
|
|
|
+ spyGetJournaledEdits(spies, 2, 21, () -> semaphore.acquireUninterruptibly(2));
|
|
|
|
|
|
List<EditLogInputStream> streams = new ArrayList<>();
|
|
|
qjm.selectInputStreams(streams, 21, true, true);
|
|
@@ -1147,17 +1144,6 @@ public class TestQuorumJournalManager {
|
|
|
assertEquals(40, streams.get(0).getLastTxId());
|
|
|
}
|
|
|
|
|
|
- private void spyGetJournaledEdits(int jnSpyIdx, long fromTxId, Runnable preHook) {
|
|
|
- Mockito.doAnswer((Answer<ListenableFuture<GetJournaledEditsResponseProto>>) invocation -> {
|
|
|
- preHook.run();
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
- ListenableFuture<GetJournaledEditsResponseProto> result =
|
|
|
- (ListenableFuture<GetJournaledEditsResponseProto>) invocation.callRealMethod();
|
|
|
- return result;
|
|
|
- }).when(spies.get(jnSpyIdx)).getJournaledEdits(fromTxId,
|
|
|
- QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
|
|
|
- }
|
|
|
-
|
|
|
@Test
|
|
|
public void testSelectViaRpcAfterJNRestart() throws Exception {
|
|
|
EditLogOutputStream stm =
|
|
@@ -1210,27 +1196,10 @@ public class TestQuorumJournalManager {
|
|
|
// expected
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- private QuorumJournalManager createSpyingQJM()
|
|
|
- throws IOException, URISyntaxException {
|
|
|
- AsyncLogger.Factory spyFactory = new AsyncLogger.Factory() {
|
|
|
- @Override
|
|
|
- public AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo,
|
|
|
- String journalId, String nameServiceId, InetSocketAddress addr) {
|
|
|
- AsyncLogger logger = new IPCLoggerChannel(conf, nsInfo, journalId,
|
|
|
- nameServiceId, addr) {
|
|
|
- protected ExecutorService createSingleThreadExecutor() {
|
|
|
- // Don't parallelize calls to the quorum in the tests.
|
|
|
- // This makes the tests more deterministic.
|
|
|
- return new DirectExecutorService();
|
|
|
- }
|
|
|
- };
|
|
|
-
|
|
|
- return Mockito.spy(logger);
|
|
|
- }
|
|
|
- };
|
|
|
- return closeLater(new QuorumJournalManager(
|
|
|
- conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO, spyFactory));
|
|
|
+
|
|
|
+ private QuorumJournalManager createSpyingQJM() throws IOException {
|
|
|
+ return closeLater(SpyQJournalUtil.createSpyingQJM(
|
|
|
+ conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO, null));
|
|
|
}
|
|
|
|
|
|
private static void waitForAllPendingCalls(AsyncLoggerSet als)
|