|
@@ -30,6 +30,8 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.ha.ClientBaseWithFixes;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
|
+import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
|
|
|
+import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
|
|
|
import org.apache.hadoop.ha.HealthMonitor;
|
|
|
import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
|
|
|
import org.apache.hadoop.ha.ZKFCTestUtil;
|
|
@@ -41,8 +43,11 @@ import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.MockNameNodeResourceChecker;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
|
import org.apache.hadoop.net.ServerSocketUtil;
|
|
|
+import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
+import org.apache.hadoop.test.LambdaTestUtils;
|
|
|
import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
|
|
|
import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
|
|
|
import org.junit.After;
|
|
@@ -201,7 +206,25 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
|
|
|
waitForHAState(0, HAServiceState.ACTIVE);
|
|
|
waitForHAState(1, HAServiceState.STANDBY);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests that a Namenode in Observer state rejects any state transition
|
|
|
+ * request from ZKFC, as a result of namenode's participation in the ZK
|
|
|
+ * election before getting transition to Observer state, since Observer
|
|
|
+ * Namenode isn't supposed to participate in ZKFC elections.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testObserverRejectZkfcCall() throws Exception {
|
|
|
+ NamenodeProtocols nn1 = cluster.getNameNode(1).getRpcServer();
|
|
|
+ nn1.transitionToObserver(
|
|
|
+ new StateChangeRequestInfo(RequestSource.REQUEST_BY_USER_FORCED));
|
|
|
+ waitForHAState(1, HAServiceState.OBSERVER);
|
|
|
+ LambdaTestUtils.intercept(AccessControlException.class,
|
|
|
+ "denied since the namenode is in Observer state.",
|
|
|
+ () -> nn1.transitionToStandby(
|
|
|
+ new StateChangeRequestInfo(RequestSource.REQUEST_BY_ZKFC)));
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout=30000)
|
|
|
public void testManualFailoverWithDFSHAAdmin() throws Exception {
|
|
|
DFSHAAdmin tool = new DFSHAAdmin();
|