|
@@ -28,7 +28,7 @@ import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
-
|
|
|
+import static org.mockito.ArgumentMatchers.eq;
|
|
|
import java.io.Closeable;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
@@ -40,11 +40,15 @@ 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 org.apache.hadoop.hdfs.server.common.Util;
|
|
|
import org.apache.hadoop.net.MockDomainNameResolver;
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture;
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.SettableFuture;
|
|
|
import org.apache.hadoop.util.Lists;
|
|
|
+import org.mockito.stubbing.Answer;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -53,6 +57,7 @@ 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;
|
|
@@ -1101,6 +1106,56 @@ public class TestQuorumJournalManager {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test selecting EditLogInputStream after some journalNode jitter.
|
|
|
+ * Suppose there are 3 journalNodes, JN0 ~ JN2.
|
|
|
+ * 1. JN0 has some abnormal cases when Active Namenode is syncing 10 Edits with first txid 11.
|
|
|
+ * 2. NameNode just ignore the abnormal JN0 and continue to sync Edits to Journal 1 and 2.
|
|
|
+ * 3. JN0 backed to health.
|
|
|
+ * 4. NameNode continue sync 10 Edits with first txid 21.
|
|
|
+ * 5. At this point, there are no Edits 11 ~ 30 in the cache of JN0.
|
|
|
+ * 6. Observer NameNode try to select EditLogInputStream through
|
|
|
+ * getJournaledEdits with since txId 21.
|
|
|
+ * 7. JN2 has some abnormal cases and caused a slow response.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testSelectViaRPCAfterJNJitter() throws Exception {
|
|
|
+ EditLogOutputStream stm = qjm.startLogSegment(
|
|
|
+ 1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
|
|
+ SettableFuture<Void> slowLog = SettableFuture.create();
|
|
|
+ Mockito.doReturn(slowLog).when(spies.get(0))
|
|
|
+ .sendEdits(eq(1L), eq(11L), eq(10), Mockito.any());
|
|
|
+ // Successfully write these edits to JN0 ~ JN2
|
|
|
+ writeTxns(stm, 1, 10);
|
|
|
+ // Failed write these edits to JN0, but successfully write them to JN1 ~ JN2
|
|
|
+ writeTxns(stm, 11, 10);
|
|
|
+ // Successfully write these edits to JN1 ~ JN2
|
|
|
+ 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));
|
|
|
+
|
|
|
+ List<EditLogInputStream> streams = new ArrayList<>();
|
|
|
+ qjm.selectInputStreams(streams, 21, true, true);
|
|
|
+
|
|
|
+ assertEquals(1, streams.size());
|
|
|
+ assertEquals(21, streams.get(0).getFirstTxId());
|
|
|
+ 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 =
|