YARN-819. ResourceManager and NodeManager should check for a minimum allowed version (Robert Parker via jeagles)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1526660 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Turner Eagles 2013-09-26 19:55:26 +00:00
parent 7e73b9cde6
commit 5a15c392a1
13 changed files with 228 additions and 4 deletions

View File

@ -36,6 +36,9 @@ Release 2.3.0 - UNRELEASED
YARN-353. Add Zookeeper-based store implementation for RMStateStore.
(Bikas Saha, Jian He and Karthik Kambatla via hitesh)
YARN-819. ResourceManager and NodeManager should check for a minimum allowed
version (Robert Parker via jeagles)
OPTIMIZATIONS
BUG FIXES

View File

@ -362,6 +362,13 @@ public class YarnConfiguration extends Configuration {
public static final long DEFAULT_RM_NMTOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS =
24 * 60 * 60;
public static final String RM_NODEMANAGER_MINIMUM_VERSION =
RM_PREFIX + "nodemanager.minimum.version";
public static final String DEFAULT_RM_NODEMANAGER_MINIMUM_VERSION =
"NONE";
////////////////////////////////
// Node Manager Configs
////////////////////////////////
@ -460,6 +467,10 @@ public class YarnConfiguration extends Configuration {
public static final String NM_LOG_DIRS = NM_PREFIX + "log-dirs";
public static final String DEFAULT_NM_LOG_DIRS = "/tmp/logs";
public static final String NM_RESOURCEMANAGER_MINIMUM_VERSION =
NM_PREFIX + "resourcemanager.minimum.version";
public static final String DEFAULT_NM_RESOURCEMANAGER_MINIMUM_VERSION = "NONE";
/** Interval at which the delayed token removal thread runs */
public static final String RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS =
RM_PREFIX + "delayed.delegation-token.removal-interval-ms";

View File

@ -358,6 +358,14 @@
<value>1000</value>
</property>
<property>
<description>The minimum allowed version of a connecting nodemanager. The valid values are
NONE (no version checking), EqualToRM (the nodemanager's version is equal to
or greater than the RM version), or a Version String.</description>
<name>yarn.resourcemanager.nodemanager.minimum.version</name>
<value>NONE</value>
</property>
<property>
<description>Enable a set of periodic monitors (specified in
yarn.resourcemanager.scheduler.monitor.policies) that affect the
@ -737,6 +745,14 @@
<value>30</value>
</property>
<property>
<description>The minimum allowed version of a resourcemanager that a nodemanager will connect to.
The valid values are NONE (no version checking), EqualToNM (the resourcemanager's version is
equal to or greater than the NM version), or a Version String.</description>
<name>yarn.nodemanager.resourcemanager.minimum.version</name>
<value>NONE</value>
</property>
<property>
<description>Max number of threads in NMClientAsync to process container
management events</description>

View File

@ -25,8 +25,10 @@ public interface RegisterNodeManagerRequest {
NodeId getNodeId();
int getHttpPort();
Resource getResource();
String getNMVersion();
void setNodeId(NodeId nodeId);
void setHttpPort(int port);
void setResource(Resource resource);
void setNMVersion(String version);
}

View File

@ -42,4 +42,7 @@ public interface RegisterNodeManagerResponse {
void setDiagnosticsMessage(String diagnosticsMessage);
void setRMVersion(String version);
String getRMVersion();
}

View File

@ -139,6 +139,21 @@ public void setHttpPort(int httpPort) {
builder.setHttpPort(httpPort);
}
@Override
public String getNMVersion() {
RegisterNodeManagerRequestProtoOrBuilder p = viaProto ? proto : builder;
if (!p.hasNmVersion()) {
return "";
}
return (p.getNmVersion());
}
@Override
public void setNMVersion(String version) {
maybeInitBuilder();
builder.setNmVersion(version);
}
private NodeIdPBImpl convertFromProtoFormat(NodeIdProto p) {
return new NodeIdPBImpl(p);
}

View File

@ -150,6 +150,25 @@ public void setDiagnosticsMessage(String diagnosticsMessage) {
builder.setDiagnosticsMessage((diagnosticsMessage));
}
@Override
public String getRMVersion() {
RegisterNodeManagerResponseProtoOrBuilder p = viaProto ? proto : builder;
if (!p.hasRmVersion()) {
return null;
}
return p.getRmVersion();
}
@Override
public void setRMVersion(String rmVersion) {
maybeInitBuilder();
if (rmVersion == null) {
builder.clearRmIdentifier();
return;
}
builder.setRmVersion(rmVersion);
}
@Override
public NodeAction getNodeAction() {
RegisterNodeManagerResponseProtoOrBuilder p = viaProto ? proto : builder;

View File

@ -29,6 +29,7 @@ message RegisterNodeManagerRequestProto {
optional NodeIdProto node_id = 1;
optional int32 http_port = 3;
optional ResourceProto resource = 4;
optional string nm_version = 5;
}
message RegisterNodeManagerResponseProto {
@ -37,6 +38,7 @@ message RegisterNodeManagerResponseProto {
optional NodeActionProto nodeAction = 3;
optional int64 rm_identifier = 4;
optional string diagnostics_message = 5;
optional string rm_version = 6;
}
message NodeHeartbeatRequestProto {
@ -45,7 +47,6 @@ message NodeHeartbeatRequestProto {
optional MasterKeyProto last_known_nm_token_master_key = 3;
}
message NodeHeartbeatResponseProto {
optional int32 response_id = 1;
optional MasterKeyProto container_token_master_key = 2;

View File

@ -37,6 +37,7 @@
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.VersionUtil;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
@ -63,6 +64,7 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.util.YarnVersionInfo;
import com.google.common.annotations.VisibleForTesting;
@ -84,6 +86,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
private ResourceTracker resourceTracker;
private Resource totalResource;
private int httpPort;
private String nodeManagerVersionId;
private String minimumResourceManagerVersion;
private volatile boolean isStopped;
private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
private boolean tokenKeepAliveEnabled;
@ -138,6 +142,10 @@ protected void serviceInit(Configuration conf) throws Exception {
this.tokenRemovalDelayMs =
conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
this.minimumResourceManagerVersion = conf.get(
YarnConfiguration.NM_RESOURCEMANAGER_MINIMUM_VERSION,
YarnConfiguration.DEFAULT_NM_RESOURCEMANAGER_MINIMUM_VERSION);
// Default duration to track stopped containers on nodemanager is 10Min.
// This should not be assigned very large value as it will remember all the
@ -168,6 +176,7 @@ protected void serviceStart() throws Exception {
// NodeManager is the last service to start, so NodeId is available.
this.nodeId = this.context.getNodeId();
this.httpPort = this.context.getHttpPort();
this.nodeManagerVersionId = YarnVersionInfo.getVersion();
try {
// Registration has to be in start so that ContainerManager can get the
// perNM tokens needed to authenticate ContainerTokens.
@ -235,6 +244,7 @@ protected void registerWithRM() throws YarnException, IOException {
request.setHttpPort(this.httpPort);
request.setResource(this.totalResource);
request.setNodeId(this.nodeId);
request.setNMVersion(this.nodeManagerVersionId);
RegisterNodeManagerResponse regNMResponse =
resourceTracker.registerNodeManager(request);
this.rmIdentifier = regNMResponse.getRMIdentifier();
@ -248,6 +258,26 @@ protected void registerWithRM() throws YarnException, IOException {
+ message);
}
// if ResourceManager version is too old then shutdown
if (!minimumResourceManagerVersion.equals("NONE")){
if (minimumResourceManagerVersion.equals("EqualToNM")){
minimumResourceManagerVersion = nodeManagerVersionId;
}
String rmVersion = regNMResponse.getRMVersion();
if (rmVersion == null) {
String message = "The Resource Manager's did not return a version. "
+ "Valid version cannot be checked.";
throw new YarnRuntimeException("Shutting down the Node Manager. "
+ message);
}
if (VersionUtil.compareVersions(rmVersion,minimumResourceManagerVersion) < 0) {
String message = "The Resource Manager's version ("
+ rmVersion +") is less than the minimum "
+ "allowed version " + minimumResourceManagerVersion;
throw new YarnRuntimeException("Shutting down the Node Manager on RM "
+ "version error, " + message);
}
}
MasterKey masterKey = regNMResponse.getContainerTokenMasterKey();
// do this now so that its set before we start heartbeating to RM
// It is expected that status updater is started by this point and

View File

@ -145,7 +145,7 @@ public static MasterKey createMasterKey() {
.byteValue() }));
return masterKey;
}
private class MyResourceTracker implements ResourceTracker {
private final Context context;
@ -471,6 +471,7 @@ private class MyResourceTracker2 implements ResourceTracker {
public NodeAction heartBeatNodeAction = NodeAction.NORMAL;
public NodeAction registerNodeAction = NodeAction.NORMAL;
public String shutDownMessage = "";
public String rmVersion = "3.0.1";
@Override
public RegisterNodeManagerResponse registerNodeManager(
@ -483,6 +484,7 @@ public RegisterNodeManagerResponse registerNodeManager(
response.setContainerTokenMasterKey(createMasterKey());
response.setNMTokenMasterKey(createMasterKey());
response.setDiagnosticsMessage(shutDownMessage);
response.setRMVersion(rmVersion);
return response;
}
@Override
@ -1180,6 +1182,44 @@ public void testNodeStatusUpdaterRetryAndNMShutdown()
" connectionWaitSecs and RetryIntervalSecs", heartBeatID == 2);
}
@Test
public void testRMVersionLessThanMinimum() throws InterruptedException {
final AtomicInteger numCleanups = new AtomicInteger(0);
YarnConfiguration conf = createNMConfig();
conf.set(YarnConfiguration.NM_RESOURCEMANAGER_MINIMUM_VERSION, "3.0.0");
nm = new NodeManager() {
@Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
MyNodeStatusUpdater myNodeStatusUpdater = new MyNodeStatusUpdater(
context, dispatcher, healthChecker, metrics);
MyResourceTracker2 myResourceTracker2 = new MyResourceTracker2();
myResourceTracker2.heartBeatNodeAction = NodeAction.NORMAL;
myResourceTracker2.rmVersion = "3.0.0";
myNodeStatusUpdater.resourceTracker = myResourceTracker2;
return myNodeStatusUpdater;
}
@Override
protected void cleanupContainers(NodeManagerEventType eventType) {
super.cleanupContainers(NodeManagerEventType.SHUTDOWN);
numCleanups.incrementAndGet();
}
};
nm.init(conf);
nm.start();
// NM takes a while to reach the STARTED state.
int waitCount = 0;
while (nm.getServiceState() != STATE.STARTED && waitCount++ != 20) {
LOG.info("Waiting for NM to stop..");
Thread.sleep(1000);
}
Assert.assertTrue(nm.getServiceState() == STATE.STARTED);
nm.stop();
}
private class MyNMContext extends NMContext {
ConcurrentMap<ContainerId, Container> containers =
new ConcurrentSkipListMap<ContainerId, Container>();

View File

@ -28,6 +28,7 @@
import org.apache.hadoop.net.Node;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.VersionUtil;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
@ -55,6 +56,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
import org.apache.hadoop.yarn.util.RackResolver;
import org.apache.hadoop.yarn.util.YarnVersionInfo;
public class ResourceTrackerService extends AbstractService implements
ResourceTracker {
@ -73,6 +75,7 @@ public class ResourceTrackerService extends AbstractService implements
private long nextHeartBeatInterval;
private Server server;
private InetSocketAddress resourceTrackerAddress;
private String minimumNodeManagerVersion;
private static final NodeHeartbeatResponse resync = recordFactory
.newRecordInstance(NodeHeartbeatResponse.class);
@ -99,6 +102,7 @@ public ResourceTrackerService(RMContext rmContext,
this.nmLivelinessMonitor = nmLivelinessMonitor;
this.containerTokenSecretManager = containerTokenSecretManager;
this.nmTokenSecretManager = nmTokenSecretManager;
}
@Override
@ -124,7 +128,11 @@ protected void serviceInit(Configuration conf) throws Exception {
minAllocVcores = conf.getInt(
YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
minimumNodeManagerVersion = conf.get(
YarnConfiguration.RM_NODEMANAGER_MINIMUM_VERSION,
YarnConfiguration.DEFAULT_RM_NODEMANAGER_MINIMUM_VERSION);
super.serviceInit(conf);
}
@ -172,10 +180,30 @@ public RegisterNodeManagerResponse registerNodeManager(
int cmPort = nodeId.getPort();
int httpPort = request.getHttpPort();
Resource capability = request.getResource();
String nodeManagerVersion = request.getNMVersion();
RegisterNodeManagerResponse response = recordFactory
.newRecordInstance(RegisterNodeManagerResponse.class);
if (!minimumNodeManagerVersion.equals("NONE")) {
if (minimumNodeManagerVersion.equals("EqualToRM")) {
minimumNodeManagerVersion = YarnVersionInfo.getVersion();
}
if ((nodeManagerVersion == null) ||
(VersionUtil.compareVersions(nodeManagerVersion,minimumNodeManagerVersion)) < 0) {
String message =
"Disallowed NodeManager Version " + nodeManagerVersion
+ ", is less than the minimum version "
+ minimumNodeManagerVersion + " sending SHUTDOWN signal to "
+ "NodeManager.";
LOG.info(message);
response.setDiagnosticsMessage(message);
response.setNodeAction(NodeAction.SHUTDOWN);
return response;
}
}
// Check if this node is a 'valid' node
if (!this.nodesListManager.isValidNode(host)) {
String message =
@ -230,6 +258,7 @@ public RegisterNodeManagerResponse registerNodeManager(
LOG.info(message);
response.setNodeAction(NodeAction.NORMAL);
response.setRMIdentifier(ResourceManager.getClusterTimeStamp());
response.setRMVersion(YarnVersionInfo.getVersion());
return response;
}

View File

@ -58,6 +58,7 @@
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.YarnVersionInfo;
import org.apache.hadoop.yarn.util.resource.Resources;
@Private
@ -96,9 +97,9 @@ public NodeManager(String hostName, int containerManagerPort, int httpPort,
RegisterNodeManagerRequest request = recordFactory
.newRecordInstance(RegisterNodeManagerRequest.class);
request.setHttpPort(httpPort);
request.setNodeId(this.nodeId);
request.setResource(capability);
request.setNodeId(this.nodeId);
request.setNMVersion(YarnVersionInfo.getVersion());
resourceTrackerService.registerNodeManager(request);
this.schedulerNode = new FiCaSchedulerNode(rmContext.getRMNodes().get(
this.nodeId), false);

View File

@ -46,6 +46,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.YarnVersionInfo;
import org.junit.After;
import org.junit.Test;
@ -248,6 +249,59 @@ public void testAddNewExcludePathToConfiguration() throws Exception {
checkDecommissionedNMCount(rm, ++initialMetricCount);
}
@Test
public void testNodeRegistrationSuccess() throws Exception {
writeToHostsFile("host2");
Configuration conf = new Configuration();
conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, hostFile
.getAbsolutePath());
rm = new MockRM(conf);
rm.start();
ResourceTrackerService resourceTrackerService = rm.getResourceTrackerService();
RegisterNodeManagerRequest req = Records.newRecord(
RegisterNodeManagerRequest.class);
NodeId nodeId = NodeId.newInstance("host2", 1234);
Resource capability = BuilderUtils.newResource(1024, 1);
req.setResource(capability);
req.setNodeId(nodeId);
req.setHttpPort(1234);
req.setNMVersion(YarnVersionInfo.getVersion());
// trying to register a invalid node.
RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(req);
Assert.assertEquals(NodeAction.NORMAL,response.getNodeAction());
}
@Test
public void testNodeRegistrationVersionLessThanRM() throws Exception {
writeToHostsFile("host2");
Configuration conf = new Configuration();
conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH, hostFile
.getAbsolutePath());
conf.set(YarnConfiguration.RM_NODEMANAGER_MINIMUM_VERSION,"EqualToRM" );
rm = new MockRM(conf);
rm.start();
String nmVersion = "1.9.9";
ResourceTrackerService resourceTrackerService = rm.getResourceTrackerService();
RegisterNodeManagerRequest req = Records.newRecord(
RegisterNodeManagerRequest.class);
NodeId nodeId = NodeId.newInstance("host2", 1234);
Resource capability = BuilderUtils.newResource(1024, 1);
req.setResource(capability);
req.setNodeId(nodeId);
req.setHttpPort(1234);
req.setNMVersion(nmVersion);
// trying to register a invalid node.
RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(req);
Assert.assertEquals(NodeAction.SHUTDOWN,response.getNodeAction());
Assert.assertTrue("Diagnostic message did not contain: 'Disallowed NodeManager " +
"Version "+ nmVersion + ", is less than the minimum version'",
response.getDiagnosticsMessage().contains("Disallowed NodeManager Version " +
nmVersion + ", is less than the minimum version "));
}
@Test
public void testNodeRegistrationFailure() throws Exception {
writeToHostsFile("host1");