YARN-1029. Added embedded leader election in the ResourceManager. Contributed by Karthik Kambatla.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1556103 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2014-01-07 01:56:11 +00:00
parent 5241aa4cdd
commit c3cc855d27
22 changed files with 779 additions and 309 deletions

View File

@ -767,7 +767,8 @@ private void createConnection() throws IOException, KeeperException {
LOG.debug("Created new connection for " + this);
}
void terminateConnection() {
@InterfaceAudience.Private
public void terminateConnection() {
if (zkClient == null) {
return;
}

View File

@ -55,6 +55,9 @@ Release 2.4.0 - UNRELEASED
YARN-1028. Added FailoverProxyProvider capability to ResourceManager to help
with RM failover. (Karthik Kambatla via vinodkv)
YARN-1029. Added embedded leader election in the ResourceManager. (Karthik
Kambatla via vinodkv)
IMPROVEMENTS
YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu)

View File

@ -51,6 +51,22 @@ public static boolean isHAEnabled(Configuration conf) {
YarnConfiguration.DEFAULT_RM_HA_ENABLED);
}
public static boolean isAutomaticFailoverEnabled(Configuration conf) {
return conf.getBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED,
YarnConfiguration.DEFAULT_AUTO_FAILOVER_ENABLED);
}
public static boolean isAutomaticFailoverEnabledAndEmbedded(
Configuration conf) {
return isAutomaticFailoverEnabled(conf) &&
isAutomaticFailoverEmbedded(conf);
}
public static boolean isAutomaticFailoverEmbedded(Configuration conf) {
return conf.getBoolean(YarnConfiguration.AUTO_FAILOVER_EMBEDDED,
YarnConfiguration.DEFAULT_AUTO_FAILOVER_EMBEDDED);
}
/**
* Verify configuration for Resource Manager HA.
* @param conf Configuration
@ -162,8 +178,7 @@ public static Collection<String> getRMHAIds(Configuration conf) {
* @param conf Configuration. Please use verifyAndSetRMHAId to check.
* @return RM Id on success
*/
@VisibleForTesting
static String getRMHAId(Configuration conf) {
public static String getRMHAId(Configuration conf) {
return conf.get(YarnConfiguration.RM_HA_ID);
}

View File

@ -87,6 +87,8 @@ public class YarnConfiguration extends Configuration {
////////////////////////////////
public static final String RM_PREFIX = "yarn.resourcemanager.";
public static final String RM_CLUSTER_ID = RM_PREFIX + "cluster-id";
/** The address of the applications manager interface in the RM.*/
public static final String RM_ADDRESS =
RM_PREFIX + "address";
@ -278,6 +280,36 @@ public class YarnConfiguration extends Configuration {
public static final String RECOVERY_ENABLED = RM_PREFIX + "recovery.enabled";
public static final boolean DEFAULT_RM_RECOVERY_ENABLED = false;
/** Zookeeper interaction configs */
public static final String RM_ZK_PREFIX = RM_PREFIX + "zk-";
public static final String RM_ZK_ADDRESS = RM_ZK_PREFIX + "address";
public static final String RM_ZK_NUM_RETRIES = RM_ZK_PREFIX + "num-retries";
public static final int DEFAULT_ZK_RM_NUM_RETRIES = 500;
public static final String RM_ZK_RETRY_INTERVAL_MS =
RM_ZK_PREFIX + "retry-interval-ms";
public static final long DEFAULT_RM_ZK_RETRY_INTERVAL_MS = 2000;
public static final String RM_ZK_TIMEOUT_MS = RM_ZK_PREFIX + "timeout-ms";
public static final int DEFAULT_RM_ZK_TIMEOUT_MS = 10000;
public static final String RM_ZK_ACL = RM_ZK_PREFIX + "acl";
public static final String DEFAULT_RM_ZK_ACL = "world:anyone:rwcda";
public static final String ZK_STATE_STORE_PREFIX =
RM_PREFIX + "zk-state-store.";
/** Parent znode path under which ZKRMStateStore will create znodes */
public static final String ZK_RM_STATE_STORE_PARENT_PATH =
ZK_STATE_STORE_PREFIX + "parent-path";
public static final String DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH = "/rmstore";
/** Root node ACLs for fencing */
public static final String ZK_RM_STATE_STORE_ROOT_NODE_ACL =
ZK_STATE_STORE_PREFIX + "root-node.acl";
/** HA related configs */
public static final String RM_HA_PREFIX = RM_PREFIX + "ha.";
public static final String RM_HA_ENABLED = RM_HA_PREFIX + "enabled";
@ -296,6 +328,22 @@ public class YarnConfiguration extends Configuration {
HttpConfig.isSecure() ? RM_WEBAPP_HTTPS_ADDRESS
: RM_WEBAPP_ADDRESS));
public static final String AUTO_FAILOVER_PREFIX =
RM_HA_PREFIX + "automatic-failover.";
public static final String AUTO_FAILOVER_ENABLED =
AUTO_FAILOVER_PREFIX + "enabled";
public static final boolean DEFAULT_AUTO_FAILOVER_ENABLED = false;
public static final String AUTO_FAILOVER_EMBEDDED =
AUTO_FAILOVER_PREFIX + "embedded";
public static final boolean DEFAULT_AUTO_FAILOVER_EMBEDDED = false;
public static final String AUTO_FAILOVER_ZK_BASE_PATH =
AUTO_FAILOVER_PREFIX + "zk-base-path";
public static final String DEFAULT_AUTO_FAILOVER_ZK_BASE_PATH =
"/yarn-leader-election";
public static final String CLIENT_FAILOVER_PREFIX =
YARN_PREFIX + "client.failover-";
public static final String CLIENT_FAILOVER_PROXY_PROVIDER =
@ -334,36 +382,6 @@ public class YarnConfiguration extends Configuration {
+ "fs.state-store.retry-policy-spec";
public static final String DEFAULT_FS_RM_STATE_STORE_RETRY_POLICY_SPEC =
"2000, 500";
/**
* Comma separated host:port pairs, each corresponding to a ZK server for
* ZKRMStateStore
*/
public static final String ZK_STATE_STORE_PREFIX =
RM_PREFIX + "zk-state-store.";
public static final String ZK_RM_STATE_STORE_NUM_RETRIES =
ZK_STATE_STORE_PREFIX + "num-retries";
public static final int DEFAULT_ZK_RM_STATE_STORE_NUM_RETRIES = 500;
/** retry interval when connecting to zookeeper*/
public static final String ZK_RM_STATE_STORE_RETRY_INTERVAL_MS =
ZK_STATE_STORE_PREFIX + "retry-interval-ms";
public static final long DEFAULT_ZK_RM_STATE_STORE_RETRY_INTERVAL_MS = 2000;
public static final String ZK_RM_STATE_STORE_ADDRESS =
ZK_STATE_STORE_PREFIX + "address";
/** Timeout in millisec for ZK server connection for ZKRMStateStore */
public static final String ZK_RM_STATE_STORE_TIMEOUT_MS =
ZK_STATE_STORE_PREFIX + "timeout-ms";
public static final int DEFAULT_ZK_RM_STATE_STORE_TIMEOUT_MS = 60000;
/** Parent znode path under which ZKRMStateStore will create znodes */
public static final String ZK_RM_STATE_STORE_PARENT_PATH =
ZK_STATE_STORE_PREFIX + "parent-path";
public static final String DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH = "/rmstore";
/** ACL for znodes in ZKRMStateStore */
public static final String ZK_RM_STATE_STORE_ACL =
ZK_STATE_STORE_PREFIX + "acl";
public static final String DEFAULT_ZK_RM_STATE_STORE_ACL =
"world:anyone:rwcda";
public static final String ZK_RM_STATE_STORE_ROOT_NODE_ACL =
ZK_STATE_STORE_PREFIX + "root-node.acl";
/** The maximum number of completed applications RM keeps. */
public static final String RM_MAX_COMPLETED_APPLICATIONS =

View File

@ -133,3 +133,11 @@ message RMStateVersionProto {
optional int32 major_version = 1;
optional int32 minor_version = 2;
}
//////////////////////////////////////////////////////////////////
///////////// RM Failover related records ////////////////////////
//////////////////////////////////////////////////////////////////
message ActiveRMInfoProto {
required string clusterid = 1;
required string rmId = 2;
}

View File

@ -30,6 +30,12 @@
</properties>
<dependencies>
<dependency>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-api</artifactId>

View File

@ -28,37 +28,39 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ha.ClientBaseWithFixes;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
import org.apache.hadoop.yarn.client.api.YarnClient;
import org.apache.hadoop.yarn.conf.HAUtil;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.MiniYARNCluster;
import org.apache.hadoop.yarn.server.resourcemanager.AdminService;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
public class TestRMFailover {
public class TestRMFailover extends ClientBaseWithFixes {
private static final Log LOG =
LogFactory.getLog(TestRMFailover.class.getName());
private static final HAServiceProtocol.StateChangeRequestInfo req =
new HAServiceProtocol.StateChangeRequestInfo(
HAServiceProtocol.RequestSource.REQUEST_BY_USER);
private static final String RM1_NODE_ID = "rm1";
private static final int RM1_PORT_BASE = 10000;
private static final String RM2_NODE_ID = "rm2";
private static final int RM2_PORT_BASE = 20000;
private static final HAServiceProtocol.StateChangeRequestInfo req =
new HAServiceProtocol.StateChangeRequestInfo(
HAServiceProtocol.RequestSource.REQUEST_BY_USER_FORCED);
private static Configuration conf;
private static MiniYARNCluster cluster;
private Configuration conf;
private MiniYARNCluster cluster;
private static void setConfForRM(String rmId, String prefix, String value) {
private void setConfForRM(String rmId, String prefix, String value) {
conf.set(HAUtil.addSuffix(prefix, rmId), value);
}
private static void setRpcAddressForRM(String rmId, int base) {
private void setRpcAddressForRM(String rmId, int base) {
setConfForRM(rmId, YarnConfiguration.RM_ADDRESS, "0.0.0.0:" +
(base + YarnConfiguration.DEFAULT_RM_PORT));
setConfForRM(rmId, YarnConfiguration.RM_SCHEDULER_ADDRESS, "0.0.0.0:" +
@ -73,13 +75,8 @@ private static void setRpcAddressForRM(String rmId, int base) {
(base + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT));
}
private static AdminService getRMAdminService(int index) {
return
cluster.getResourceManager(index).getRMContext().getRMAdminService();
}
@BeforeClass
public static void setup() throws IOException {
@Before
public void setup() throws IOException {
conf = new YarnConfiguration();
conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
conf.set(YarnConfiguration.RM_HA_IDS, RM1_NODE_ID + "," + RM2_NODE_ID);
@ -87,27 +84,22 @@ public static void setup() throws IOException {
setRpcAddressForRM(RM2_NODE_ID, RM2_PORT_BASE);
conf.setLong(YarnConfiguration.CLIENT_FAILOVER_SLEEPTIME_BASE_MS, 100L);
conf.setBoolean(YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS, true);
conf.setBoolean(YarnConfiguration.YARN_MINICLUSTER_USE_RPC, true);
cluster = new MiniYARNCluster(TestRMFailover.class.getName(), 2, 1, 1, 1);
cluster.init(conf);
cluster.start();
cluster.getResourceManager(0).getRMContext().getRMAdminService()
.transitionToActive(req);
assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
}
@AfterClass
public static void teardown() {
@After
public void teardown() {
cluster.stop();
}
private void verifyClientConnection() {
int numRetries = 3;
while(numRetries-- > 0) {
Configuration conf = new YarnConfiguration(TestRMFailover.conf);
Configuration conf = new YarnConfiguration(this.conf);
YarnClient client = YarnClient.createYarnClient();
client.init(conf);
client.start();
@ -123,31 +115,68 @@ private void verifyClientConnection() {
fail("Client couldn't connect to the Active RM");
}
private void verifyConnections() throws InterruptedException, YarnException {
assertTrue("NMs failed to connect to the RM",
cluster.waitForNodeManagersToConnect(20000));
verifyClientConnection();
}
private AdminService getAdminService(int index) {
return cluster.getResourceManager(index).getRMContext().getRMAdminService();
}
private void explicitFailover() throws IOException {
int activeRMIndex = cluster.getActiveRMIndex();
int newActiveRMIndex = (activeRMIndex + 1) % 2;
getAdminService(activeRMIndex).transitionToStandby(req);
getAdminService(newActiveRMIndex).transitionToActive(req);
assertEquals("Failover failed", newActiveRMIndex, cluster.getActiveRMIndex());
}
private void failover()
throws IOException, InterruptedException, YarnException {
int activeRMIndex = cluster.getActiveRMIndex();
cluster.stopResourceManager(activeRMIndex);
assertEquals("Failover failed",
(activeRMIndex + 1) % 2, cluster.getActiveRMIndex());
cluster.restartResourceManager(activeRMIndex);
}
@Test
public void testExplicitFailover()
throws YarnException, InterruptedException, IOException {
assertTrue("NMs failed to connect to the RM",
cluster.waitForNodeManagersToConnect(5000));
verifyClientConnection();
conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
cluster.init(conf);
cluster.start();
getAdminService(0).transitionToActive(req);
assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
verifyConnections();
// Failover to the second RM
getRMAdminService(0).transitionToStandby(req);
getRMAdminService(1).transitionToActive(req);
assertEquals("Wrong ResourceManager is active",
HAServiceProtocol.HAServiceState.ACTIVE,
getRMAdminService(1).getServiceStatus().getState());
assertTrue("NMs failed to connect to the RM",
cluster.waitForNodeManagersToConnect(5000));
verifyClientConnection();
explicitFailover();
verifyConnections();
// Failover back to the first RM
getRMAdminService(1).transitionToStandby(req);
getRMAdminService(0).transitionToActive(req);
assertEquals("Wrong ResourceManager is active",
HAServiceProtocol.HAServiceState.ACTIVE,
getRMAdminService(0).getServiceStatus().getState());
assertTrue("NMs failed to connect to the RM",
cluster.waitForNodeManagersToConnect(5000));
verifyClientConnection();
explicitFailover();
verifyConnections();
}
@Test
public void testAutomaticFailover()
throws YarnException, InterruptedException, IOException {
conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, true);
conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_EMBEDDED, true);
conf.set(YarnConfiguration.RM_CLUSTER_ID, "yarn-test-cluster");
conf.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
conf.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, 2000);
cluster.init(conf);
cluster.start();
assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
verifyConnections();
failover();
verifyConnections();
failover();
verifyConnections();
}
}

View File

@ -21,16 +21,19 @@
import org.apache.hadoop.ha.BadFencingConfigurationException;
import org.apache.hadoop.ha.HAServiceTarget;
import org.apache.hadoop.ha.NodeFencer;
import org.apache.hadoop.yarn.conf.HAUtil;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import java.io.IOException;
import java.net.InetSocketAddress;
public class RMHAServiceTarget extends HAServiceTarget {
private InetSocketAddress haAdminServiceAddress;
private final boolean autoFailoverEnabled;
private final InetSocketAddress haAdminServiceAddress;
public RMHAServiceTarget(YarnConfiguration conf)
throws IOException {
autoFailoverEnabled = HAUtil.isAutomaticFailoverEnabled(conf);
haAdminServiceAddress = conf.getSocketAddr(
YarnConfiguration.RM_ADMIN_ADDRESS,
YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
@ -44,19 +47,23 @@ public InetSocketAddress getAddress() {
@Override
public InetSocketAddress getZKFCAddress() {
// TODO (YARN-1177): Hook up ZKFC information
return null;
// TODO (YARN-1177): ZKFC implementation
throw new UnsupportedOperationException("RMHAServiceTarget doesn't have " +
"a corresponding ZKFC address");
}
@Override
public NodeFencer getFencer() {
// TODO (YARN-1026): Hook up fencing implementation
return null;
}
@Override
public void checkFencingConfigured()
throws BadFencingConfigurationException {
// TODO (YARN-1026): Based on fencing implementation
public void checkFencingConfigured() throws BadFencingConfigurationException {
throw new BadFencingConfigurationException("Fencer not configured");
}
@Override
public boolean isAutoFailoverEnabled() {
return autoFailoverEnabled;
}
}

View File

@ -264,13 +264,20 @@
<property>
<description>Enable RM to recover state after starting. If true, then
yarn.resourcemanager.store.class must be specified</description>
yarn.resourcemanager.store.class must be specified. </description>
<name>yarn.resourcemanager.recovery.enabled</name>
<value>false</value>
</property>
<property>
<description>The class to use as the persistent store.</description>
<description>The class to use as the persistent store.
If org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
is used, the store is implicitly fenced; meaning a single ResourceManager
is able to use the store at any point in time. More details on this
implicit fencing, along with setting up appropriate ACLs is discussed
under yarn.resourcemanager.zk-state-store.root-node.acl.
</description>
<name>yarn.resourcemanager.store.class</name>
<value>org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore</value>
</property>
@ -291,31 +298,24 @@
</property>
<property>
<description>Host:Port of the ZooKeeper server where RM state will
be stored. This must be supplied when using
org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
as the value for yarn.resourcemanager.store.class. ZKRMStateStore
is implicitly fenced, meaning a single ResourceManager is
able to use the store at any point in time. More details on this, along
with setting up appropriate ACLs is discussed under the description for
yarn.resourcemanager.zk-state-store.root-node.acl.</description>
<name>yarn.resourcemanager.zk-state-store.address</name>
<description>Host:Port of the ZooKeeper server to be used by the RM. This
must be supplied when using the ZooKeeper based implementation of the
RM state store and/or embedded automatic failover in a HA setting.
</description>
<name>yarn.resourcemanager.zk-address</name>
<!--value>127.0.0.1:2181</value-->
</property>
<property>
<description>Number of times ZKRMStateStore tries to connect to
ZooKeeper. This may be supplied when using
org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
as the value for yarn.resourcemanager.store.class</description>
<name>yarn.resourcemanager.zk-state-store.num-retries</name>
<description>Number of times RM tries to connect to ZooKeeper.</description>
<name>yarn.resourcemanager.zk-num-retries</name>
<value>500</value>
</property>
<property>
<description>Retry interval in milliseconds when ZKRMStateStore tries to
connect to ZooKeeper.</description>
<name>yarn.resourcemanager.zk-state-store.retry-interval-ms</name>
<description>Retry interval in milliseconds when connecting to ZooKeeper.
</description>
<name>yarn.resourcemanager.zk-retry-interval-ms</name>
<value>2000</value>
</property>
@ -333,20 +333,14 @@
is managed by the ZooKeeper cluster itself, not by the client. This value is
used by the cluster to determine when the client's session expires.
Expirations happens when the cluster does not hear from the client within
the specified session timeout period (i.e. no heartbeat).
This may be supplied when using
org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
as the value for yarn.resourcemanager.store.class</description>
<name>yarn.resourcemanager.zk-state-store.timeout-ms</name>
the specified session timeout period (i.e. no heartbeat).</description>
<name>yarn.resourcemanager.zk-timeout-ms</name>
<value>60000</value>
</property>
<property>
<description>ACL's to be used for ZooKeeper znodes.
This may be supplied when using
org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
as the value for yarn.resourcemanager.store.class</description>
<name>yarn.resourcemanager.zk-state-store.acl</name>
<description>ACL's to be used for ZooKeeper znodes.</description>
<name>yarn.resourcemanager.zk-acl</name>
<value>world:anyone:rwcda</value>
</property>
@ -362,7 +356,7 @@
permissions.
By default, when this property is not set, we use the ACLs from
yarn.resourcemanager.zk-state-store.acl for shared admin access and
yarn.resourcemanager.zk-acl for shared admin access and
rm-address:cluster-timestamp for username-based exclusive create-delete
access.
@ -408,6 +402,36 @@
<value>false</value>
</property>
<property>
<description>Enable automatic failover.</description>
<name>yarn.resourcemanager.ha.automatic-failover.enabled</name>
<value>false</value>
</property>
<property>
<description>Enable embedded automatic failover. The embedded elector
relies on the RM state store to handle fencing, and is primarily intended
to be used in conjunction with ZKRMStateStore.</description>
<name>yarn.resourcemanager.ha.automatic-failover.embedded</name>
<value>false</value>
</property>
<property>
<description>The base znode path to use for storing leader information,
when using ZooKeeper based leader election.</description>
<name>yarn.resourcemanager.ha.automatic-failover.zk-base-path</name>
<value>/yarn-leader-election</value>
</property>
<property>
<description>Name of the cluster. In a HA setting,
this is used to ensure the RM participates in leader
election fo this cluster and ensures it does not affect
other clusters</description>
<name>yarn.resourcemanager.cluster-id</name>
<!--value>yarn-cluster</value-->
</property>
<property>
<description>The list of RM nodes in the cluster when HA is
enabled. See description of yarn.resourcemanager.ha

View File

@ -20,11 +20,8 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -41,14 +38,16 @@
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RPC.Server;
import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.Groups;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.ProxyUsers;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.conf.HAUtil;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.factories.RecordFactory;
@ -75,7 +74,7 @@
import com.google.protobuf.BlockingService;
public class AdminService extends AbstractService implements
public class AdminService extends CompositeService implements
HAServiceProtocol, ResourceManagerAdministrationProtocol {
private static final Log LOG = LogFactory.getLog(AdminService.class);
@ -84,6 +83,8 @@ public class AdminService extends AbstractService implements
private final ResourceManager rm;
private String rmId;
private boolean autoFailoverEnabled;
private Server server;
private InetSocketAddress masterServiceAddress;
private AccessControlList adminAcl;
@ -99,6 +100,15 @@ public AdminService(ResourceManager rm, RMContext rmContext) {
@Override
public synchronized void serviceInit(Configuration conf) throws Exception {
if (rmContext.isHAEnabled()) {
autoFailoverEnabled = HAUtil.isAutomaticFailoverEnabled(conf);
if (autoFailoverEnabled) {
if (HAUtil.isAutomaticFailoverEmbedded(conf)) {
addIfService(createEmbeddedElectorService());
}
}
}
masterServiceAddress = conf.getSocketAddr(
YarnConfiguration.RM_ADMIN_ADDRESS,
YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
@ -162,6 +172,10 @@ protected void stopServer() throws Exception {
}
}
protected EmbeddedElectorService createEmbeddedElectorService() {
return new EmbeddedElectorService(rmContext);
}
private UserGroupInformation checkAccess(String method) throws IOException {
return RMServerUtils.verifyAccess(adminAcl, method, LOG);
}
@ -174,6 +188,43 @@ private UserGroupInformation checkAcls(String method) throws YarnException {
}
}
/**
* Check that a request to change this node's HA state is valid.
* In particular, verifies that, if auto failover is enabled, non-forced
* requests from the HAAdmin CLI are rejected, and vice versa.
*
* @param req the request to check
* @throws AccessControlException if the request is disallowed
*/
private void checkHaStateChange(StateChangeRequestInfo req)
throws AccessControlException {
switch (req.getSource()) {
case REQUEST_BY_USER:
if (autoFailoverEnabled) {
throw new AccessControlException(
"Manual failover for this ResourceManager is disallowed, " +
"because automatic failover is enabled.");
}
break;
case REQUEST_BY_USER_FORCED:
if (autoFailoverEnabled) {
LOG.warn("Allowing manual failover from " +
org.apache.hadoop.ipc.Server.getRemoteAddress() +
" even though automatic failover is enabled, because the user " +
"specified the force flag");
}
break;
case REQUEST_BY_ZKFC:
if (!autoFailoverEnabled) {
throw new AccessControlException(
"Request from ZK failover controller at " +
org.apache.hadoop.ipc.Server.getRemoteAddress() + " denied " +
"since automatic failover is not enabled");
}
break;
}
}
private synchronized boolean isRMActive() {
return HAServiceState.ACTIVE == rmContext.getHAServiceState();
}
@ -196,8 +247,7 @@ public synchronized void monitorHealth()
public synchronized void transitionToActive(
HAServiceProtocol.StateChangeRequestInfo reqInfo) throws IOException {
UserGroupInformation user = checkAccess("transitionToActive");
// TODO (YARN-1177): When automatic failover is enabled,
// check if transition should be allowed for this request
checkHaStateChange(reqInfo);
try {
rm.transitionToActive();
RMAuditLogger.logSuccess(user.getShortUserName(),
@ -215,8 +265,7 @@ public synchronized void transitionToActive(
public synchronized void transitionToStandby(
HAServiceProtocol.StateChangeRequestInfo reqInfo) throws IOException {
UserGroupInformation user = checkAccess("transitionToStandby");
// TODO (YARN-1177): When automatic failover is enabled,
// check if transition should be allowed for this request
checkHaStateChange(reqInfo);
try {
rm.transitionToStandby(true);
RMAuditLogger.logSuccess(user.getShortUserName(),

View File

@ -0,0 +1,209 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ha.ActiveStandbyElector;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.ServiceFailedException;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ZKUtil;
import org.apache.hadoop.yarn.conf.HAUtil;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.ACL;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class EmbeddedElectorService extends AbstractService
implements ActiveStandbyElector.ActiveStandbyElectorCallback {
private static final Log LOG =
LogFactory.getLog(EmbeddedElectorService.class.getName());
private static final HAServiceProtocol.StateChangeRequestInfo req =
new HAServiceProtocol.StateChangeRequestInfo(
HAServiceProtocol.RequestSource.REQUEST_BY_ZKFC);
private RMContext rmContext;
private byte[] localActiveNodeInfo;
private ActiveStandbyElector elector;
EmbeddedElectorService(RMContext rmContext) {
super(EmbeddedElectorService.class.getName());
this.rmContext = rmContext;
}
@Override
protected synchronized void serviceInit(Configuration conf)
throws Exception {
conf = conf instanceof YarnConfiguration ? conf : new YarnConfiguration(conf);
String zkQuorum = conf.get(YarnConfiguration.RM_ZK_ADDRESS);
if (zkQuorum == null) {
throw new YarnRuntimeException("Embedded automatic failover " +
"is enabled, but " + YarnConfiguration.RM_ZK_ADDRESS +
" is not set");
}
String rmId = HAUtil.getRMHAId(conf);
String clusterId = conf.get(YarnConfiguration.RM_CLUSTER_ID);
if (clusterId == null) {
throw new YarnRuntimeException(YarnConfiguration.RM_CLUSTER_ID +
" is not specified!");
}
localActiveNodeInfo = createActiveNodeInfo(clusterId, rmId);
String zkBasePath = conf.get(YarnConfiguration.AUTO_FAILOVER_ZK_BASE_PATH,
YarnConfiguration.DEFAULT_AUTO_FAILOVER_ZK_BASE_PATH);
String electionZNode = zkBasePath + "/" + clusterId;
long zkSessionTimeout = conf.getLong(YarnConfiguration.RM_ZK_TIMEOUT_MS,
YarnConfiguration.DEFAULT_RM_ZK_TIMEOUT_MS);
String zkAclConf = conf.get(YarnConfiguration.RM_ZK_ACL,
YarnConfiguration.DEFAULT_RM_ZK_ACL);
List<ACL> zkAcls;
try {
zkAcls = ZKUtil.parseACLs(ZKUtil.resolveConfIndirection(zkAclConf));
} catch (ZKUtil.BadAclFormatException bafe) {
throw new YarnRuntimeException(
YarnConfiguration.RM_ZK_ACL + "has ill-formatted ACLs");
}
// TODO (YARN-1528): ZKAuthInfo to be set for rm-store and elector
List<ZKUtil.ZKAuthInfo> zkAuths = Collections.emptyList();
elector = new ActiveStandbyElector(zkQuorum, (int) zkSessionTimeout,
electionZNode, zkAcls, zkAuths, this);
elector.ensureParentZNode();
if (!isParentZnodeSafe(clusterId)) {
notifyFatalError(electionZNode + " znode has invalid data! "+
"Might need formatting!");
}
super.serviceInit(conf);
}
@Override
protected synchronized void serviceStart() throws Exception {
elector.joinElection(localActiveNodeInfo);
super.serviceStart();
}
@Override
protected synchronized void serviceStop() throws Exception {
elector.quitElection(false);
elector.terminateConnection();
super.serviceStop();
}
@Override
public synchronized void becomeActive() throws ServiceFailedException {
try {
rmContext.getRMAdminService().transitionToActive(req);
} catch (Exception e) {
throw new ServiceFailedException("RM could not transition to Active", e);
}
}
@Override
public synchronized void becomeStandby() {
try {
rmContext.getRMAdminService().transitionToStandby(req);
} catch (Exception e) {
LOG.error("RM could not transition to Standby", e);
}
}
@Override
public void enterNeutralMode() {
/**
* Possibly due to transient connection issues. Do nothing.
* TODO: Might want to keep track of how long in this state and transition
* to standby.
*/
}
@SuppressWarnings(value = "unchecked")
@Override
public synchronized void notifyFatalError(String errorMessage) {
rmContext.getDispatcher().getEventHandler().handle(
new RMFatalEvent(RMFatalEventType.EMBEDDED_ELECTOR_FAILED, errorMessage));
}
@Override
public synchronized void fenceOldActive(byte[] oldActiveData) {
if (LOG.isDebugEnabled()) {
LOG.debug("Request to fence old active being ignored, " +
"as embedded leader election doesn't support fencing");
}
}
private static byte[] createActiveNodeInfo(String clusterId, String rmId)
throws IOException {
return YarnServerResourceManagerServiceProtos.ActiveRMInfoProto
.newBuilder()
.setClusterid(clusterId)
.setRmId(rmId)
.build()
.toByteArray();
}
private synchronized boolean isParentZnodeSafe(String clusterId)
throws InterruptedException, IOException, KeeperException {
byte[] data;
try {
data = elector.getActiveData();
} catch (ActiveStandbyElector.ActiveNotFoundException e) {
// no active found, parent znode is safe
return true;
}
YarnServerResourceManagerServiceProtos.ActiveRMInfoProto proto;
try {
proto = YarnServerResourceManagerServiceProtos.ActiveRMInfoProto
.parseFrom(data);
} catch (InvalidProtocolBufferException e) {
LOG.error("Invalid data in ZK: " + StringUtils.byteToHexString(data));
return false;
}
// Check if the passed proto corresponds to an RM in the same cluster
if (!proto.getClusterid().equals(clusterId)) {
LOG.error("Mismatched cluster! The other RM seems " +
"to be from a different cluster. Current cluster = " + clusterId +
"Other RM's cluster = " + proto.getClusterid());
return false;
}
return true;
}
}

View File

@ -0,0 +1,37 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.event.AbstractEvent;
public class RMFatalEvent extends AbstractEvent<RMFatalEventType> {
private String cause;
public RMFatalEvent(RMFatalEventType rmFatalEventType, String cause) {
super(rmFatalEventType);
this.cause = cause;
}
public RMFatalEvent(RMFatalEventType rmFatalEventType, Exception cause) {
super(rmFatalEventType);
this.cause = StringUtils.stringifyException(cause);
}
public String getCause() {return this.cause;}
}

View File

@ -15,9 +15,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
public enum RMStateStoreOperationFailedEventType {
FENCED, // Store operation failed because it was fenced
FAILED // Store operation failed for no known reason
package org.apache.hadoop.yarn.server.resourcemanager;
import org.apache.hadoop.classification.InterfaceAudience;
@InterfaceAudience.Private
public enum RMFatalEventType {
// Source <- Store
STATE_STORE_FENCED,
STATE_STORE_OP_FAILED,
// Source <- Embedded Elector
EMBEDDED_ELECTOR_FAILED
}

View File

@ -59,8 +59,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreFactory;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreOperationFailedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreOperationFailedEventType;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@ -121,6 +119,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
*/
@VisibleForTesting
protected RMContextImpl rmContext;
private Dispatcher rmDispatcher;
@VisibleForTesting
protected AdminService adminService;
@ -134,7 +133,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
*/
protected RMActiveServices activeServices;
protected RMSecretManagerService rmSecretManagerService;
private Dispatcher rmDispatcher;
protected ResourceScheduler scheduler;
private ClientRMService clientRM;
@ -179,6 +177,13 @@ protected void serviceInit(Configuration conf) throws Exception {
this.conf = conf;
this.rmContext = new RMContextImpl();
rmDispatcher = createDispatcher();
addIfService(rmDispatcher);
rmContext.setDispatcher(rmDispatcher);
rmDispatcher.register(RMFatalEventType.class,
new ResourceManager.RMFatalEventDispatcher(this.rmContext, this));
adminService = createAdminService();
addService(adminService);
rmContext.setRMAdminService(adminService);
@ -207,11 +212,6 @@ protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
return new SchedulerEventDispatcher(this.scheduler);
}
protected RMStateStoreOperationFailedEventDispatcher
createRMStateStoreOperationFailedEventDispatcher() {
return new RMStateStoreOperationFailedEventDispatcher(rmContext, this);
}
protected Dispatcher createDispatcher() {
return new AsyncDispatcher();
}
@ -297,10 +297,6 @@ class RMActiveServices extends CompositeService {
protected void serviceInit(Configuration configuration) throws Exception {
conf.setBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY, true);
rmDispatcher = createDispatcher();
addIfService(rmDispatcher);
rmContext.setDispatcher(rmDispatcher);
rmSecretManagerService = createRMSecretManagerService();
addService(rmSecretManagerService);
@ -332,8 +328,6 @@ protected void serviceInit(Configuration configuration) throws Exception {
try {
rmStore.init(conf);
rmStore.setRMDispatcher(rmDispatcher);
rmDispatcher.register(RMStateStoreOperationFailedEventType.class,
createRMStateStoreOperationFailedEventDispatcher());
} catch (Exception e) {
// the Exception from stateStore.init() needs to be handled for
// HA and we need to give up master status if we got fenced
@ -605,26 +599,23 @@ public void handle(SchedulerEvent event) {
}
@Private
public static class RMStateStoreOperationFailedEventDispatcher implements
EventHandler<RMStateStoreOperationFailedEvent> {
public static class RMFatalEventDispatcher
implements EventHandler<RMFatalEvent> {
private final RMContext rmContext;
private final ResourceManager rm;
public RMStateStoreOperationFailedEventDispatcher(RMContext rmContext,
ResourceManager resourceManager) {
public RMFatalEventDispatcher(
RMContext rmContext, ResourceManager resourceManager) {
this.rmContext = rmContext;
this.rm = resourceManager;
}
@Override
public void handle(RMStateStoreOperationFailedEvent event) {
if (LOG.isDebugEnabled()) {
LOG.debug("Received a " +
RMStateStoreOperationFailedEvent.class.getName() + " of type " +
public void handle(RMFatalEvent event) {
LOG.fatal("Received a " + RMFatalEvent.class.getName() + " of type " +
event.getType().name());
}
if (event.getType() == RMStateStoreOperationFailedEventType.FENCED) {
if (event.getType() == RMFatalEventType.STATE_STORE_FENCED) {
LOG.info("RMStateStore has been fenced");
if (rmContext.isHAEnabled()) {
try {
@ -633,14 +624,11 @@ public void handle(RMStateStoreOperationFailedEvent event) {
rm.transitionToStandby(true);
return;
} catch (Exception e) {
LOG.error("Failed to transition RM to Standby mode.");
LOG.fatal("Failed to transition RM to Standby mode.");
}
}
}
LOG.error("Shutting down RM on receiving a " +
RMStateStoreOperationFailedEvent.class.getName() + " of type " +
event.getType().name());
ExitUtil.terminate(1, event.getCause());
}
}

View File

@ -48,6 +48,8 @@
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
import org.apache.hadoop.yarn.server.resourcemanager.RMFatalEvent;
import org.apache.hadoop.yarn.server.resourcemanager.RMFatalEventType;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMStateVersion;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
@ -679,15 +681,13 @@ protected void handleStoreEvent(RMStateStoreEvent event) {
* @param failureCause the exception due to which the operation failed
*/
private void notifyStoreOperationFailed(Exception failureCause) {
RMStateStoreOperationFailedEventType type;
RMFatalEventType type;
if (failureCause instanceof StoreFencedException) {
type = RMStateStoreOperationFailedEventType.FENCED;
type = RMFatalEventType.STATE_STORE_FENCED;
} else {
type = RMStateStoreOperationFailedEventType.FAILED;
type = RMFatalEventType.STATE_STORE_OP_FAILED;
}
rmDispatcher.getEventHandler().handle(
new RMStateStoreOperationFailedEvent(type, failureCause));
rmDispatcher.getEventHandler().handle(new RMFatalEvent(type, failureCause));
}
@SuppressWarnings("unchecked")

View File

@ -1,36 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
import org.apache.hadoop.yarn.event.AbstractEvent;
public class RMStateStoreOperationFailedEvent
extends AbstractEvent<RMStateStoreOperationFailedEventType> {
private Exception cause;
RMStateStoreOperationFailedEvent(
RMStateStoreOperationFailedEventType type, Exception cause) {
super(type);
this.cause = cause;
}
public Exception getCause() {
return this.cause;
}
}

View File

@ -182,34 +182,34 @@ protected List<ACL> constructZkRootNodeACL(
@Override
public synchronized void initInternal(Configuration conf) throws Exception {
zkHostPort = conf.get(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS);
zkHostPort = conf.get(YarnConfiguration.RM_ZK_ADDRESS);
if (zkHostPort == null) {
throw new YarnRuntimeException("No server address specified for " +
"zookeeper state store for Resource Manager recovery. " +
YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS + " is not configured.");
YarnConfiguration.RM_ZK_ADDRESS + " is not configured.");
}
numRetries =
conf.getInt(YarnConfiguration.ZK_RM_STATE_STORE_NUM_RETRIES,
YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_NUM_RETRIES);
conf.getInt(YarnConfiguration.RM_ZK_NUM_RETRIES,
YarnConfiguration.DEFAULT_ZK_RM_NUM_RETRIES);
znodeWorkingPath =
conf.get(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH,
YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_PARENT_PATH);
zkSessionTimeout =
conf.getInt(YarnConfiguration.ZK_RM_STATE_STORE_TIMEOUT_MS,
YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_TIMEOUT_MS);
conf.getInt(YarnConfiguration.RM_ZK_TIMEOUT_MS,
YarnConfiguration.DEFAULT_RM_ZK_TIMEOUT_MS);
zkRetryInterval =
conf.getLong(YarnConfiguration.ZK_RM_STATE_STORE_RETRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_RETRY_INTERVAL_MS);
conf.getLong(YarnConfiguration.RM_ZK_RETRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_ZK_RETRY_INTERVAL_MS);
// Parse authentication from configuration.
String zkAclConf =
conf.get(YarnConfiguration.ZK_RM_STATE_STORE_ACL,
YarnConfiguration.DEFAULT_ZK_RM_STATE_STORE_ACL);
conf.get(YarnConfiguration.RM_ZK_ACL,
YarnConfiguration.DEFAULT_RM_ZK_ACL);
zkAclConf = ZKUtil.resolveConfIndirection(zkAclConf);
try {
zkAcl = ZKUtil.parseACLs(zkAclConf);
} catch (ZKUtil.BadAclFormatException bafe) {
LOG.error("Invalid format for " + YarnConfiguration.ZK_RM_STATE_STORE_ACL);
LOG.error("Invalid format for " + YarnConfiguration.RM_ZK_ACL);
throw bafe;
}

View File

@ -396,6 +396,11 @@ protected void startServer() {
protected void stopServer() {
// don't do anything
}
@Override
protected EmbeddedElectorService createEmbeddedElectorService() {
return null;
}
};
}

View File

@ -25,6 +25,7 @@
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
import org.apache.hadoop.ha.HealthCheckFailedException;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.conf.HAUtil;
import org.junit.Before;
@ -39,6 +40,7 @@
public class TestRMHA {
private Log LOG = LogFactory.getLog(TestRMHA.class);
private final Configuration configuration = new YarnConfiguration();
private MockRM rm = null;
private static final String STATE_ERR =
"ResourceManager is in wrong HA state";
@ -51,17 +53,13 @@ public class TestRMHA {
@Before
public void setUp() throws Exception {
Configuration conf = new YarnConfiguration();
conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
conf.set(YarnConfiguration.RM_HA_IDS, RM1_NODE_ID + "," + RM2_NODE_ID);
configuration.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
configuration.set(YarnConfiguration.RM_HA_IDS, RM1_NODE_ID + "," + RM2_NODE_ID);
for (String confKey : YarnConfiguration.RM_SERVICES_ADDRESS_CONF_KEYS) {
conf.set(HAUtil.addSuffix(confKey, RM1_NODE_ID), RM1_ADDRESS);
conf.set(HAUtil.addSuffix(confKey, RM2_NODE_ID), RM2_ADDRESS);
configuration.set(HAUtil.addSuffix(confKey, RM1_NODE_ID), RM1_ADDRESS);
configuration.set(HAUtil.addSuffix(confKey, RM2_NODE_ID), RM2_ADDRESS);
}
conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID);
rm = new MockRM(conf);
rm.init(conf);
configuration.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID);
}
private void checkMonitorHealth() throws IOException {
@ -113,6 +111,9 @@ private void checkActiveRMFunctionality() throws IOException {
*/
@Test (timeout = 30000)
public void testStartAndTransitions() throws IOException {
Configuration conf = new YarnConfiguration(configuration);
rm = new MockRM(conf);
rm.init(conf);
StateChangeRequestInfo requestInfo = new StateChangeRequestInfo(
HAServiceProtocol.RequestSource.REQUEST_BY_USER);
@ -162,4 +163,63 @@ public void testStartAndTransitions() throws IOException {
rm.areActiveServicesRunning());
checkMonitorHealth();
}
@Test
public void testTransitionsWhenAutomaticFailoverEnabled() throws IOException {
final String ERR_UNFORCED_REQUEST = "User request succeeded even when " +
"automatic failover is enabled";
Configuration conf = new YarnConfiguration(configuration);
conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, true);
rm = new MockRM(conf);
rm.init(conf);
rm.start();
StateChangeRequestInfo requestInfo = new StateChangeRequestInfo(
HAServiceProtocol.RequestSource.REQUEST_BY_USER);
// Transition to standby
try {
rm.adminService.transitionToStandby(requestInfo);
fail(ERR_UNFORCED_REQUEST);
} catch (AccessControlException e) {
// expected
}
checkMonitorHealth();
checkStandbyRMFunctionality();
// Transition to active
try {
rm.adminService.transitionToActive(requestInfo);
fail(ERR_UNFORCED_REQUEST);
} catch (AccessControlException e) {
// expected
}
checkMonitorHealth();
checkStandbyRMFunctionality();
final String ERR_FORCED_REQUEST = "Forced request by user should work " +
"even if automatic failover is enabled";
requestInfo = new StateChangeRequestInfo(
HAServiceProtocol.RequestSource.REQUEST_BY_USER_FORCED);
// Transition to standby
try {
rm.adminService.transitionToStandby(requestInfo);
} catch (AccessControlException e) {
fail(ERR_FORCED_REQUEST);
}
checkMonitorHealth();
checkStandbyRMFunctionality();
// Transition to active
try {
rm.adminService.transitionToActive(requestInfo);
} catch (AccessControlException e) {
fail(ERR_FORCED_REQUEST);
}
checkMonitorHealth();
checkActiveRMFunctionality();
}
}

View File

@ -92,7 +92,7 @@ public String getAppNode(String appId) {
public RMStateStore getRMStateStore() throws Exception {
YarnConfiguration conf = new YarnConfiguration();
workingZnode = "/Test";
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
conf.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
this.client = createClient();
this.store = new TestZKRMStateStoreInternal(conf, workingZnode);
@ -140,7 +140,7 @@ private Configuration createHARMConf(
conf.set(YarnConfiguration.RM_HA_IDS, rmIds);
conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
conf.set(YarnConfiguration.RM_STORE, ZKRMStateStore.class.getName());
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
conf.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
conf.set(YarnConfiguration.RM_HA_ID, rmId);
for (String rpcAddress : YarnConfiguration.RM_SERVICES_ADDRESS_CONF_KEYS) {
for (String id : HAUtil.getRMHAIds(conf)) {

View File

@ -107,7 +107,7 @@ public void process(WatchedEvent event) {
public RMStateStore getRMStateStore(Configuration conf) throws Exception {
String workingZnode = "/Test";
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
conf.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
watcher = new TestForwardingWatcher();
this.store = new TestZKRMStateStore(conf, workingZnode);
@ -120,8 +120,8 @@ public void testZKClientRetry() throws Exception {
TestZKClient zkClientTester = new TestZKClient();
final String path = "/test";
YarnConfiguration conf = new YarnConfiguration();
conf.setInt(YarnConfiguration.ZK_RM_STATE_STORE_TIMEOUT_MS, 1000);
conf.setLong(YarnConfiguration.ZK_RM_STATE_STORE_RETRY_INTERVAL_MS, 100);
conf.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, 1000);
conf.setLong(YarnConfiguration.RM_ZK_RETRY_INTERVAL_MS, 100);
final ZKRMStateStore store =
(ZKRMStateStore) zkClientTester.getRMStateStore(conf);
TestDispatcher dispatcher = new TestDispatcher();
@ -153,7 +153,7 @@ public void testZKClientDisconnectAndReconnect()
TestZKClient zkClientTester = new TestZKClient();
String path = "/test";
YarnConfiguration conf = new YarnConfiguration();
conf.setInt(YarnConfiguration.ZK_RM_STATE_STORE_TIMEOUT_MS, 100);
conf.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, 100);
ZKRMStateStore store =
(ZKRMStateStore) zkClientTester.getRMStateStore(conf);
TestDispatcher dispatcher = new TestDispatcher();
@ -195,7 +195,7 @@ public void testZKSessionTimeout() throws Exception {
TestZKClient zkClientTester = new TestZKClient();
String path = "/test";
YarnConfiguration conf = new YarnConfiguration();
conf.setInt(YarnConfiguration.ZK_RM_STATE_STORE_TIMEOUT_MS, 100);
conf.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, 100);
ZKRMStateStore store =
(ZKRMStateStore) zkClientTester.getRMStateStore(conf);
TestDispatcher dispatcher = new TestDispatcher();
@ -227,7 +227,7 @@ public void testZKSessionTimeout() throws Exception {
public void testSetZKAcl() {
TestZKClient zkClientTester = new TestZKClient();
YarnConfiguration conf = new YarnConfiguration();
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ACL, "world:anyone:rwca");
conf.set(YarnConfiguration.RM_ZK_ACL, "world:anyone:rwca");
try {
zkClientTester.store.zkClient.delete(zkClientTester.store
.znodeWorkingPath, -1);
@ -240,7 +240,7 @@ public void testSetZKAcl() {
public void testInvalidZKAclConfiguration() {
TestZKClient zkClientTester = new TestZKClient();
YarnConfiguration conf = new YarnConfiguration();
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ACL, "randomstring&*");
conf.set(YarnConfiguration.RM_ZK_ACL, "randomstring&*");
try {
zkClientTester.getRMStateStore(conf);
fail("ZKRMStateStore created with bad ACL");

View File

@ -58,6 +58,7 @@
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
import org.apache.hadoop.yarn.server.resourcemanager.RMFatalEvent;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
@ -98,6 +99,7 @@ public class MiniYARNCluster extends CompositeService {
private boolean useFixedPorts;
private boolean useRpc = false;
private int failoverTimeout;
private ConcurrentMap<ApplicationAttemptId, Long> appMasters =
new ConcurrentHashMap<ApplicationAttemptId, Long>(16, 0.75f, 2);
@ -189,12 +191,15 @@ public void serviceInit(Configuration conf) throws Exception {
YarnConfiguration.DEFAULT_YARN_MINICLUSTER_FIXED_PORTS);
useRpc = conf.getBoolean(YarnConfiguration.YARN_MINICLUSTER_USE_RPC,
YarnConfiguration.DEFAULT_YARN_MINICLUSTER_USE_RPC);
failoverTimeout = conf.getInt(YarnConfiguration.RM_ZK_TIMEOUT_MS,
YarnConfiguration.DEFAULT_RM_ZK_TIMEOUT_MS);
if (useRpc && !useFixedPorts) {
throw new YarnRuntimeException("Invalid configuration!" +
" Minicluster can use rpc only when configured to use fixed ports");
}
conf.setBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, true);
if (resourceManagers.length > 1) {
conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
if (conf.get(YarnConfiguration.RM_HA_IDS) == null) {
@ -218,6 +223,13 @@ protected void doSecureLogin() throws IOException {
// Don't try to login using keytab in the testcases.
}
};
if (!useFixedPorts) {
if (HAUtil.isHAEnabled(conf)) {
setHARMConfiguration(i, conf);
} else {
setNonHARMConfiguration(conf);
}
}
addService(new ResourceManagerWrapper(i));
}
for(int index = 0; index < nodeManagers.length; index++) {
@ -230,18 +242,103 @@ protected void doSecureLogin() throws IOException {
conf instanceof YarnConfiguration ? conf : new YarnConfiguration(conf));
}
private void setNonHARMConfiguration(Configuration conf) {
String hostname = MiniYARNCluster.getHostname();
conf.set(YarnConfiguration.RM_ADDRESS, hostname + ":0");
conf.set(YarnConfiguration.RM_ADMIN_ADDRESS, hostname + ":0");
conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, hostname + ":0");
conf.set(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS, hostname + ":0");
WebAppUtils.setRMWebAppHostnameAndPort(conf, hostname, 0);
}
private void setHARMConfiguration(final int index, Configuration conf) {
String hostname = MiniYARNCluster.getHostname();
for (String confKey : YarnConfiguration.RM_SERVICES_ADDRESS_CONF_KEYS) {
conf.set(HAUtil.addSuffix(confKey, rmIds[index]), hostname + ":0");
}
}
private synchronized void initResourceManager(int index, Configuration conf) {
if (HAUtil.isHAEnabled(conf)) {
conf.set(YarnConfiguration.RM_HA_ID, rmIds[index]);
}
resourceManagers[index].init(conf);
resourceManagers[index].getRMContext().getDispatcher().register(
RMAppAttemptEventType.class,
new EventHandler<RMAppAttemptEvent>() {
public void handle(RMAppAttemptEvent event) {
if (event instanceof RMAppAttemptRegistrationEvent) {
appMasters.put(event.getApplicationAttemptId(),
event.getTimestamp());
} else if (event instanceof RMAppAttemptUnregistrationEvent) {
appMasters.remove(event.getApplicationAttemptId());
}
}
});
}
private synchronized void startResourceManager(final int index) {
try {
Thread rmThread = new Thread() {
public void run() {
resourceManagers[index].start();
}
};
rmThread.setName("RM-" + index);
rmThread.start();
int waitCount = 0;
while (resourceManagers[index].getServiceState() == STATE.INITED
&& waitCount++ < 60) {
LOG.info("Waiting for RM to start...");
Thread.sleep(1500);
}
if (resourceManagers[index].getServiceState() != STATE.STARTED) {
// RM could have failed.
throw new IOException(
"ResourceManager failed to start. Final state is "
+ resourceManagers[index].getServiceState());
}
} catch (Throwable t) {
throw new YarnRuntimeException(t);
}
LOG.info("MiniYARN ResourceManager address: " +
getConfig().get(YarnConfiguration.RM_ADDRESS));
LOG.info("MiniYARN ResourceManager web address: " +
WebAppUtils.getRMWebAppURLWithoutScheme(getConfig()));
}
@InterfaceAudience.Private
@VisibleForTesting
public synchronized void stopResourceManager(int index) {
if (resourceManagers[index] != null) {
resourceManagers[index].stop();
resourceManagers[index] = null;
}
}
@InterfaceAudience.Private
@VisibleForTesting
public synchronized void restartResourceManager(int index)
throws InterruptedException {
if (resourceManagers[index] != null) {
resourceManagers[index].stop();
resourceManagers[index] = null;
}
Configuration conf = getConfig();
resourceManagers[index] = new ResourceManager();
initResourceManager(index, getConfig());
startResourceManager(index);
}
public File getTestWorkDir() {
return testWorkDir;
}
/**
* In a HA cluster, go through all the RMs and find the Active RM. If none
* of them are active, wait upto 5 seconds for them to transition to Active.
* In a HA cluster, go through all the RMs and find the Active RM. In a
* non-HA cluster, return the index of the only RM.
*
* In an non-HA cluster, return the index of the only RM.
*
* @return index of the active RM or -1 if none of them transition to
* active even after 5 seconds of waiting
* @return index of the active RM or -1 if none of them turn active
*/
@InterfaceAudience.Private
@VisibleForTesting
@ -250,9 +347,12 @@ public int getActiveRMIndex() {
return 0;
}
int numRetriesForRMBecomingActive = 5;
int numRetriesForRMBecomingActive = failoverTimeout / 100;
while (numRetriesForRMBecomingActive-- > 0) {
for (int i = 0; i < resourceManagers.length; i++) {
if (resourceManagers[i] == null) {
continue;
}
try {
if (HAServiceProtocol.HAServiceState.ACTIVE ==
resourceManagers[i].getRMContext().getRMAdminService()
@ -265,7 +365,7 @@ public int getActiveRMIndex() {
}
}
try {
Thread.sleep(1000);
Thread.sleep(100);
} catch (InterruptedException e) {
throw new YarnRuntimeException("Interrupted while waiting for one " +
"of the ResourceManagers to become active");
@ -282,7 +382,7 @@ public ResourceManager getResourceManager() {
int activeRMIndex = getActiveRMIndex();
return activeRMIndex == -1
? null
: this.resourceManagers[getActiveRMIndex()];
: this.resourceManagers[activeRMIndex];
}
public ResourceManager getResourceManager(int i) {
@ -310,82 +410,21 @@ public ResourceManagerWrapper(int i) {
index = i;
}
private void setNonHARMConfiguration(Configuration conf) {
String hostname = MiniYARNCluster.getHostname();
conf.set(YarnConfiguration.RM_ADDRESS, hostname + ":0");
conf.set(YarnConfiguration.RM_ADMIN_ADDRESS, hostname + ":0");
conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, hostname + ":0");
conf.set(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS, hostname + ":0");
WebAppUtils.setRMWebAppHostnameAndPort(conf, hostname, 0);
}
private void setHARMConfiguration(Configuration conf) {
String hostname = MiniYARNCluster.getHostname();
for (String confKey : YarnConfiguration.RM_SERVICES_ADDRESS_CONF_KEYS) {
for (String id : HAUtil.getRMHAIds(conf)) {
conf.set(HAUtil.addSuffix(confKey, id), hostname + ":0");
}
}
}
@Override
protected synchronized void serviceInit(Configuration conf)
throws Exception {
conf.setBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, true);
if (!useFixedPorts) {
if (HAUtil.isHAEnabled(conf)) {
setHARMConfiguration(conf);
} else {
setNonHARMConfiguration(conf);
}
}
if (HAUtil.isHAEnabled(conf)) {
conf.set(YarnConfiguration.RM_HA_ID, rmIds[index]);
}
resourceManagers[index].init(conf);
resourceManagers[index].getRMContext().getDispatcher().register
(RMAppAttemptEventType.class,
new EventHandler<RMAppAttemptEvent>() {
public void handle(RMAppAttemptEvent event) {
if (event instanceof RMAppAttemptRegistrationEvent) {
appMasters.put(event.getApplicationAttemptId(), event.getTimestamp());
} else if (event instanceof RMAppAttemptUnregistrationEvent) {
appMasters.remove(event.getApplicationAttemptId());
}
}
});
initResourceManager(index, conf);
super.serviceInit(conf);
}
@Override
protected synchronized void serviceStart() throws Exception {
try {
new Thread() {
public void run() {
resourceManagers[index].start();
}
}.start();
int waitCount = 0;
while (resourceManagers[index].getServiceState() == STATE.INITED
&& waitCount++ < 60) {
LOG.info("Waiting for RM to start...");
Thread.sleep(1500);
}
if (resourceManagers[index].getServiceState() != STATE.STARTED) {
// RM could have failed.
throw new IOException(
"ResourceManager failed to start. Final state is "
+ resourceManagers[index].getServiceState());
}
super.serviceStart();
} catch (Throwable t) {
throw new YarnRuntimeException(t);
}
startResourceManager(index);
LOG.info("MiniYARN ResourceManager address: " +
getConfig().get(YarnConfiguration.RM_ADDRESS));
LOG.info("MiniYARN ResourceManager web address: " +
WebAppUtils.getRMWebAppURLWithoutScheme(getConfig()));
super.serviceStart();
}
private void waitForAppMastersToFinish(long timeoutMillis) throws InterruptedException {
@ -406,7 +445,6 @@ protected synchronized void serviceStop() throws Exception {
waitForAppMastersToFinish(5000);
resourceManagers[index].stop();
}
super.serviceStop();
if (Shell.WINDOWS) {
// On Windows, clean up the short temporary symlink that was created to
@ -420,6 +458,7 @@ protected synchronized void serviceStop() throws Exception {
testWorkDir.getAbsolutePath());
}
}
super.serviceStop();
}
}