HDFS-14961. [SBN read] Prevent ZKFC changing Observer Namenode state. Contributed by Ayush Saxena.

This commit is contained in:
Ayush Saxena 2019-11-28 18:28:20 +05:30
parent c3659f8f94
commit 46166bd8d1
2 changed files with 37 additions and 3 deletions

View File

@ -1793,11 +1793,22 @@ public synchronized void transitionToActive(StateChangeRequestInfo req)
nn.checkHaStateChange(req); nn.checkHaStateChange(req);
nn.transitionToActive(); nn.transitionToActive();
} }
@Override // HAServiceProtocol @Override // HAServiceProtocol
public synchronized void transitionToStandby(StateChangeRequestInfo req) public synchronized void transitionToStandby(StateChangeRequestInfo req)
throws ServiceFailedException, AccessControlException, IOException { throws ServiceFailedException, AccessControlException, IOException {
checkNNStartup(); checkNNStartup();
// This is to eliminate any race condition between manual transition of
// namenode into Observer, and ZKFC auto failover election, when the
// namenode has already participated in the
// ZKFC election, before transition to Observer state as Standby Node.
// For more details check : HDFS-14961.
if (nn.getState().equals(NameNode.OBSERVER_STATE.toString())
&& req.getSource() == RequestSource.REQUEST_BY_ZKFC) {
throw new AccessControlException(
"Request from ZK failover controller at " + Server.getRemoteAddress()
+ " denied since the namenode is in Observer state.");
}
nn.checkHaStateChange(req); nn.checkHaStateChange(req);
nn.transitionToStandby(); nn.transitionToStandby();
} }

View File

@ -30,6 +30,8 @@
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.ha.ClientBaseWithFixes; import org.apache.hadoop.ha.ClientBaseWithFixes;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; 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.HealthMonitor;
import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer; import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
import org.apache.hadoop.ha.ZKFCTestUtil; import org.apache.hadoop.ha.ZKFCTestUtil;
@ -41,8 +43,11 @@
import org.apache.hadoop.hdfs.server.namenode.MockNameNodeResourceChecker; import org.apache.hadoop.hdfs.server.namenode.MockNameNodeResourceChecker;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil; 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.net.ServerSocketUtil;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.test.GenericTestUtils; 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.TestContext;
import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread; import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
import org.junit.After; import org.junit.After;
@ -201,7 +206,25 @@ public void testManualFailover() throws Exception {
waitForHAState(0, HAServiceState.ACTIVE); waitForHAState(0, HAServiceState.ACTIVE);
waitForHAState(1, HAServiceState.STANDBY); 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) @Test(timeout=30000)
public void testManualFailoverWithDFSHAAdmin() throws Exception { public void testManualFailoverWithDFSHAAdmin() throws Exception {
DFSHAAdmin tool = new DFSHAAdmin(); DFSHAAdmin tool = new DFSHAAdmin();