YARN-8409. Fixed NPE in ActiveStandbyElectorBasedElectorService.
Contributed by Chandni Singh
This commit is contained in:
parent
d3fa83a44b
commit
384764cdea
@ -329,9 +329,12 @@ public synchronized boolean parentZNodeExists()
|
||||
* This recursively creates the znode as well as all of its parents.
|
||||
*/
|
||||
public synchronized void ensureParentZNode()
|
||||
throws IOException, InterruptedException {
|
||||
throws IOException, InterruptedException, KeeperException {
|
||||
Preconditions.checkState(!wantToBeInElection,
|
||||
"ensureParentZNode() may not be called while in the election");
|
||||
if (zkClient == null) {
|
||||
createConnection();
|
||||
}
|
||||
|
||||
String pathParts[] = znodeWorkingDir.split("/");
|
||||
Preconditions.checkArgument(pathParts.length >= 1 &&
|
||||
|
@ -269,7 +269,7 @@ private void printUsage() {
|
||||
}
|
||||
|
||||
private int formatZK(boolean force, boolean interactive)
|
||||
throws IOException, InterruptedException {
|
||||
throws IOException, InterruptedException, KeeperException {
|
||||
if (elector.parentZNodeExists()) {
|
||||
if (!force && (!interactive || !confirmFormat())) {
|
||||
return ERR_CODE_FORMAT_DENIED;
|
||||
|
@ -22,8 +22,10 @@
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ha.ClientBaseWithFixes;
|
||||
import org.apache.hadoop.ha.ServiceFailedException;
|
||||
import org.apache.hadoop.service.ServiceStateException;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
@ -304,6 +306,26 @@ private void testCallbackSynchronizationTimingStandby(AdminService as,
|
||||
verify(as, times(1)).transitionToStandby(any());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that active elector service triggers a fatal RM Event when connection
|
||||
* to ZK fails. YARN-8409
|
||||
*/
|
||||
@Test
|
||||
public void testFailureToConnectToZookeeper() throws Exception {
|
||||
stopServer();
|
||||
Configuration myConf = new Configuration(conf);
|
||||
ResourceManager rm = new MockRM(conf);
|
||||
|
||||
ActiveStandbyElectorBasedElectorService ees =
|
||||
new ActiveStandbyElectorBasedElectorService(rm);
|
||||
try {
|
||||
ees.init(myConf);
|
||||
Assert.fail("expect failure to connect to Zookeeper");
|
||||
} catch (ServiceStateException sse) {
|
||||
Assert.assertTrue(sse.getMessage().contains("ConnectionLoss"));
|
||||
}
|
||||
}
|
||||
|
||||
private class MockRMWithElector extends MockRM {
|
||||
private long delayMs = 0;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user