|
@@ -21,6 +21,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_EN
|
|
import static org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getServiceState;
|
|
import static org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getServiceState;
|
|
import static org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider.*;
|
|
import static org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider.*;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
|
|
+import static org.junit.Assert.assertNotNull;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.fail;
|
|
import static org.junit.Assert.fail;
|
|
import static org.mockito.ArgumentMatchers.any;
|
|
import static org.mockito.ArgumentMatchers.any;
|
|
@@ -36,7 +37,10 @@ import java.util.ArrayList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.concurrent.ExecutionException;
|
|
import java.util.concurrent.ExecutionException;
|
|
import java.util.concurrent.ExecutorService;
|
|
import java.util.concurrent.ExecutorService;
|
|
|
|
+import java.util.concurrent.Executors;
|
|
import java.util.concurrent.Future;
|
|
import java.util.concurrent.Future;
|
|
|
|
+import java.util.concurrent.ScheduledExecutorService;
|
|
|
|
+import java.util.concurrent.ScheduledFuture;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -61,9 +65,12 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer;
|
|
import org.apache.hadoop.hdfs.server.namenode.TestFsck;
|
|
import org.apache.hadoop.hdfs.server.namenode.TestFsck;
|
|
import org.apache.hadoop.hdfs.tools.GetGroups;
|
|
import org.apache.hadoop.hdfs.tools.GetGroups;
|
|
import org.apache.hadoop.ipc.ObserverRetryOnActiveException;
|
|
import org.apache.hadoop.ipc.ObserverRetryOnActiveException;
|
|
|
|
+import org.apache.hadoop.ipc.metrics.RpcMetrics;
|
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
|
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
@@ -124,6 +131,43 @@ public class TestObserverNode {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testObserverRequeue() throws Exception {
|
|
|
|
+ ScheduledExecutorService interruptor =
|
|
|
|
+ Executors.newScheduledThreadPool(1);
|
|
|
|
+
|
|
|
|
+ FSNamesystem observerFsNS = dfsCluster.getNamesystem(2);
|
|
|
|
+ RpcMetrics obRpcMetrics = ((NameNodeRpcServer)dfsCluster
|
|
|
|
+ .getNameNodeRpc(2)).getClientRpcServer().getRpcMetrics();
|
|
|
|
+ try {
|
|
|
|
+ // Stop EditlogTailer of Observer NameNode.
|
|
|
|
+ observerFsNS.getEditLogTailer().stop();
|
|
|
|
+ long oldRequeueNum = obRpcMetrics.getRpcRequeueCalls();
|
|
|
|
+ ScheduledFuture<FileStatus> scheduledFuture = interruptor.schedule(
|
|
|
|
+ () -> {
|
|
|
|
+ Path tmpTestPath = new Path("/TestObserverRequeue");
|
|
|
|
+ dfs.create(tmpTestPath, (short)1).close();
|
|
|
|
+ assertSentTo(0);
|
|
|
|
+ // This operation will be blocked in ObserverNameNode
|
|
|
|
+ // until EditlogTailer tailed edits from journalNode.
|
|
|
|
+ FileStatus fileStatus = dfs.getFileStatus(tmpTestPath);
|
|
|
|
+ assertSentTo(2);
|
|
|
|
+ return fileStatus;
|
|
|
|
+ }, 0, TimeUnit.SECONDS);
|
|
|
|
+
|
|
|
|
+ GenericTestUtils.waitFor(() -> obRpcMetrics.getRpcRequeueCalls() > oldRequeueNum,
|
|
|
|
+ 50, 10000);
|
|
|
|
+
|
|
|
|
+ observerFsNS.getEditLogTailer().doTailEdits();
|
|
|
|
+ FileStatus fileStatus = scheduledFuture.get(10000, TimeUnit.MILLISECONDS);
|
|
|
|
+ assertNotNull(fileStatus);
|
|
|
|
+ } finally {
|
|
|
|
+ EditLogTailer editLogTailer = new EditLogTailer(observerFsNS, conf);
|
|
|
|
+ observerFsNS.setEditLogTailerForTests(editLogTailer);
|
|
|
|
+ editLogTailer.start();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
@Test
|
|
@Test
|
|
public void testNoActiveToObserver() throws Exception {
|
|
public void testNoActiveToObserver() throws Exception {
|
|
try {
|
|
try {
|