merge YARN-39 from trunk. RM-NM secret-keys should be randomly generated and rolled every so often. (Contributed by Vinod Kumar Vavilapalli and Siddharth Seth)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1377181 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Siddharth Seth 2012-08-25 02:25:12 +00:00
parent 5998a179b4
commit bf08346d3c
69 changed files with 1208 additions and 474 deletions

View File

@ -93,7 +93,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.BuilderUtils;
import org.junit.After;
import org.junit.Test;
@ -1099,8 +1098,7 @@ public MyFifoScheduler(RMContext rmContext) {
super();
try {
Configuration conf = new Configuration();
reinitialize(conf, new ContainerTokenSecretManager(conf),
rmContext);
reinitialize(conf, rmContext);
} catch (IOException ie) {
LOG.info("add application failed with ", ie);
assert (false);

View File

@ -61,3 +61,6 @@ Release 0.23.3 - Unreleased
refreshing of queues (Arun Murthy via tgraves)
MAPREDUCE-4323. NM leaks filesystems (Jason Lowe via jeagles)
YARN-39. RM-NM secret-keys should be randomly generated and rolled every
so often. (vinodkv and sseth via sseth)

View File

@ -18,16 +18,16 @@
package org.apache.hadoop.yarn.conf;
import com.google.common.base.Joiner;
import com.google.common.base.Splitter;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.Iterator;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.net.NetUtils;
import com.google.common.base.Joiner;
import com.google.common.base.Splitter;
public class YarnConfiguration extends Configuration {
private static final Splitter ADDR_SPLITTER = Splitter.on(':').trimResults();
private static final Joiner JOINER = Joiner.on("");
@ -262,6 +262,12 @@ public class YarnConfiguration extends Configuration {
public static final long DEFAULT_RM_APP_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS =
24 * 60 * 60;
public static final String RM_CONTAINER_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS =
RM_PREFIX + "container-tokens.master-key-rolling-interval-secs";
public static final long DEFAULT_RM_CONTAINER_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS =
24 * 60 * 60;
////////////////////////////////
// Node Manager Configs
////////////////////////////////

View File

@ -50,13 +50,15 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
private String nmHostAddr;
private Resource resource;
private long expiryTimeStamp;
private int masterKeyId;
public ContainerTokenIdentifier(ContainerId containerID, String hostName,
Resource r, long expiryTimeStamp) {
Resource r, long expiryTimeStamp, int masterKeyId) {
this.containerId = containerID;
this.nmHostAddr = hostName;
this.resource = r;
this.expiryTimeStamp = expiryTimeStamp;
this.masterKeyId = masterKeyId;
}
/**
@ -81,6 +83,10 @@ public long getExpiryTimeStamp() {
return this.expiryTimeStamp;
}
public int getMasterKeyId() {
return this.masterKeyId;
}
@Override
public void write(DataOutput out) throws IOException {
LOG.debug("Writing ContainerTokenIdentifier to RPC layer: " + this);
@ -94,6 +100,7 @@ public void write(DataOutput out) throws IOException {
out.writeUTF(this.nmHostAddr);
out.writeInt(this.resource.getMemory());
out.writeLong(this.expiryTimeStamp);
out.writeInt(this.masterKeyId);
}
@Override
@ -107,6 +114,7 @@ public void readFields(DataInput in) throws IOException {
this.nmHostAddr = in.readUTF();
this.resource = BuilderUtils.newResource(in.readInt());
this.expiryTimeStamp = in.readLong();
this.masterKeyId = in.readInt();
}
@Override

View File

@ -239,6 +239,17 @@
<value>86400</value>
</property>
<property>
<description>Interval for the roll over for the master key used to generate
container tokens. It is expected to be much greater than
yarn.nm.liveness-monitor.expiry-interval-ms and
yarn.rm.container-allocation.expiry-interval-ms. Otherwise the
behavior is undefined.
</description>
<name>yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs</name>
<value>86400</value>
</property>
<!-- Node Manager Configs -->
<property>
<description>The address of the container manager in the NM.</description>

View File

@ -37,6 +37,9 @@ public interface HeartbeatResponse {
void setResponseId(int responseId);
void setNodeAction(NodeAction action);
MasterKey getMasterKey();
void setMasterKey(MasterKey secretKey);
void addAllContainersToCleanup(List<ContainerId> containers);
void addContainerToCleanup(ContainerId container);
void removeContainerToCleanup(int index);

View File

@ -0,0 +1,33 @@
/**
* 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.api.records;
import java.nio.ByteBuffer;
public interface MasterKey {
int getKeyId();
void setKeyId(int keyId);
ByteBuffer getBytes();
void setBytes(ByteBuffer bytes);
}

View File

@ -17,14 +17,13 @@
*/
package org.apache.hadoop.yarn.server.api.records;
import java.nio.ByteBuffer;
public interface RegistrationResponse {
public abstract ByteBuffer getSecretKey();
public abstract void setSecretKey(ByteBuffer secretKey);
MasterKey getMasterKey();
public abstract NodeAction getNodeAction();
void setMasterKey(MasterKey secretKey);
public abstract void setNodeAction(NodeAction nodeAction);
NodeAction getNodeAction();
void setNodeAction(NodeAction nodeAction);
}

View File

@ -32,8 +32,10 @@
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.HeartbeatResponseProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.HeartbeatResponseProtoOrBuilder;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto;
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeAction;
public class HeartbeatResponsePBImpl extends
@ -43,9 +45,8 @@ public class HeartbeatResponsePBImpl extends
boolean viaProto = false;
private List<ContainerId> containersToCleanup = null;
private List<ApplicationId> applicationsToCleanup = null;
private MasterKey masterKey = null;
public HeartbeatResponsePBImpl() {
builder = HeartbeatResponseProto.newBuilder();
@ -71,6 +72,9 @@ private void mergeLocalToBuilder() {
if (this.applicationsToCleanup != null) {
addApplicationsToCleanupToProto();
}
if (this.masterKey != null) {
builder.setMasterKey(convertToProtoFormat(this.masterKey));
}
}
private void mergeLocalToProto() {
@ -100,6 +104,28 @@ public void setResponseId(int responseId) {
maybeInitBuilder();
builder.setResponseId((responseId));
}
@Override
public MasterKey getMasterKey() {
HeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
if (this.masterKey != null) {
return this.masterKey;
}
if (!p.hasMasterKey()) {
return null;
}
this.masterKey = convertFromProtoFormat(p.getMasterKey());
return this.masterKey;
}
@Override
public void setMasterKey(MasterKey masterKey) {
maybeInitBuilder();
if (masterKey == null)
builder.clearMasterKey();
this.masterKey = masterKey;
}
@Override
public NodeAction getNodeAction() {
HeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
@ -313,4 +339,12 @@ private NodeAction convertFromProtoFormat(NodeActionProto p) {
private NodeActionProto convertToProtoFormat(NodeAction t) {
return NodeActionProto.valueOf(t.name());
}
private MasterKeyPBImpl convertFromProtoFormat(MasterKeyProto p) {
return new MasterKeyPBImpl(p);
}
private MasterKeyProto convertToProtoFormat(MasterKey t) {
return ((MasterKeyPBImpl)t).getProto();
}
}

View File

@ -0,0 +1,102 @@
/**
* 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.api.records.impl.pb;
import java.nio.ByteBuffer;
import org.apache.hadoop.yarn.api.records.ProtoBase;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProtoOrBuilder;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
public class MasterKeyPBImpl extends ProtoBase<MasterKeyProto> implements
MasterKey {
MasterKeyProto proto = MasterKeyProto.getDefaultInstance();
MasterKeyProto.Builder builder = null;
boolean viaProto = false;
public MasterKeyPBImpl() {
builder = MasterKeyProto.newBuilder();
}
public MasterKeyPBImpl(MasterKeyProto proto) {
this.proto = proto;
viaProto = true;
}
public synchronized MasterKeyProto getProto() {
proto = viaProto ? proto : builder.build();
viaProto = true;
return proto;
}
private synchronized void maybeInitBuilder() {
if (viaProto || builder == null) {
builder = MasterKeyProto.newBuilder(proto);
}
viaProto = false;
}
@Override
public synchronized int getKeyId() {
MasterKeyProtoOrBuilder p = viaProto ? proto : builder;
return (p.getKeyId());
}
@Override
public synchronized void setKeyId(int id) {
maybeInitBuilder();
builder.setKeyId((id));
}
@Override
public synchronized ByteBuffer getBytes() {
MasterKeyProtoOrBuilder p = viaProto ? proto : builder;
return convertFromProtoFormat(p.getBytes());
}
@Override
public synchronized void setBytes(ByteBuffer bytes) {
maybeInitBuilder();
builder.setBytes(convertToProtoFormat(bytes));
}
@Override
public int hashCode() {
return getKeyId();
}
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (!(obj instanceof MasterKey)) {
return false;
}
MasterKey other = (MasterKey) obj;
if (this.getKeyId() != other.getKeyId()) {
return false;
}
if (!this.getBytes().equals(other.getBytes())) {
return false;
}
return true;
}
}

View File

@ -19,12 +19,12 @@
package org.apache.hadoop.yarn.server.api.records.impl.pb;
import java.nio.ByteBuffer;
import org.apache.hadoop.yarn.api.records.ProtoBase;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.RegistrationResponseProto;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.RegistrationResponseProtoOrBuilder;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeAction;
import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
@ -34,7 +34,7 @@ public class RegistrationResponsePBImpl extends
RegistrationResponseProto.Builder builder = null;
boolean viaProto = false;
private ByteBuffer secretKey = null;
private MasterKey masterKey = null;
public RegistrationResponsePBImpl() {
builder = RegistrationResponseProto.newBuilder();
@ -54,8 +54,8 @@ public RegistrationResponseProto getProto() {
}
private void mergeLocalToBuilder() {
if (this.secretKey != null) {
builder.setSecretKey(convertToProtoFormat(this.secretKey));
if (this.masterKey != null) {
builder.setMasterKey(convertToProtoFormat(this.masterKey));
}
}
@ -76,24 +76,24 @@ private void maybeInitBuilder() {
}
@Override
public ByteBuffer getSecretKey() {
public MasterKey getMasterKey() {
RegistrationResponseProtoOrBuilder p = viaProto ? proto : builder;
if (this.secretKey != null) {
return this.secretKey;
if (this.masterKey != null) {
return this.masterKey;
}
if (!p.hasSecretKey()) {
if (!p.hasMasterKey()) {
return null;
}
this.secretKey = convertFromProtoFormat(p.getSecretKey());
return this.secretKey;
this.masterKey = convertFromProtoFormat(p.getMasterKey());
return this.masterKey;
}
@Override
public void setSecretKey(ByteBuffer secretKey) {
public void setMasterKey(MasterKey masterKey) {
maybeInitBuilder();
if (secretKey == null)
builder.clearSecretKey();
this.secretKey = secretKey;
if (masterKey == null)
builder.clearMasterKey();
this.masterKey = masterKey;
}
@Override
@ -123,4 +123,11 @@ private NodeActionProto convertToProtoFormat(NodeAction t) {
return NodeActionProto.valueOf(t.name());
}
private MasterKeyPBImpl convertFromProtoFormat(MasterKeyProto p) {
return new MasterKeyPBImpl(p);
}
private MasterKeyProto convertToProtoFormat(MasterKey t) {
return ((MasterKeyPBImpl)t).getProto();
}
}

View File

@ -0,0 +1,202 @@
/**
* 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.security;
import java.nio.ByteBuffer;
import java.security.SecureRandom;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import javax.crypto.SecretKey;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerToken;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.util.BuilderUtils;
import org.apache.hadoop.yarn.util.Records;
/**
* SecretManager for ContainerTokens. Extended by both RM and NM and hence is
* present in yarn-server-common package.
*
*/
public class BaseContainerTokenSecretManager extends
SecretManager<ContainerTokenIdentifier> {
private static Log LOG = LogFactory
.getLog(BaseContainerTokenSecretManager.class);
private int serialNo = new SecureRandom().nextInt();
protected final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
protected final Lock readLock = readWriteLock.readLock();
protected final Lock writeLock = readWriteLock.writeLock();
/**
* THE masterKey. ResourceManager should persist this and recover it on
* restart instead of generating a new key. The NodeManagers get it from the
* ResourceManager and use it for validating container-tokens.
*/
protected MasterKeyData currentMasterKey;
protected final class MasterKeyData {
private final MasterKey masterKeyRecord;
// Underlying secret-key also stored to avoid repetitive encoding and
// decoding the masterKeyRecord bytes.
private final SecretKey generatedSecretKey;
private MasterKeyData() {
this.masterKeyRecord = Records.newRecord(MasterKey.class);
this.masterKeyRecord.setKeyId(serialNo++);
this.generatedSecretKey = generateSecret();
this.masterKeyRecord.setBytes(ByteBuffer.wrap(generatedSecretKey
.getEncoded()));
}
public MasterKeyData(MasterKey masterKeyRecord) {
this.masterKeyRecord = masterKeyRecord;
this.generatedSecretKey =
SecretManager.createSecretKey(this.masterKeyRecord.getBytes().array()
.clone());
}
public MasterKey getMasterKey() {
return this.masterKeyRecord;
}
private SecretKey getSecretKey() {
return this.generatedSecretKey;
}
}
protected final long containerTokenExpiryInterval;
public BaseContainerTokenSecretManager(Configuration conf) {
this.containerTokenExpiryInterval =
conf.getInt(YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS);
}
// Need lock as we increment serialNo etc.
protected MasterKeyData createNewMasterKey() {
this.writeLock.lock();
try {
return new MasterKeyData();
} finally {
this.writeLock.unlock();
}
}
@Private
public MasterKey getCurrentKey() {
this.readLock.lock();
try {
return this.currentMasterKey.getMasterKey();
} finally {
this.readLock.unlock();
}
}
@Override
public byte[] createPassword(ContainerTokenIdentifier identifier) {
if (LOG.isDebugEnabled()) {
LOG.debug("Creating password for " + identifier.getContainerID()
+ " to be run on NM " + identifier.getNmHostAddress());
}
this.readLock.lock();
try {
return createPassword(identifier.getBytes(),
this.currentMasterKey.getSecretKey());
} finally {
this.readLock.unlock();
}
}
@Override
public byte[] retrievePassword(ContainerTokenIdentifier identifier)
throws SecretManager.InvalidToken {
this.readLock.lock();
try {
return retrievePasswordInternal(identifier, this.currentMasterKey);
} finally {
this.readLock.unlock();
}
}
protected byte[] retrievePasswordInternal(ContainerTokenIdentifier identifier,
MasterKeyData masterKey)
throws org.apache.hadoop.security.token.SecretManager.InvalidToken {
if (LOG.isDebugEnabled()) {
LOG.debug("Retrieving password for " + identifier.getContainerID()
+ " to be run on NM " + identifier.getNmHostAddress());
}
return createPassword(identifier.getBytes(), masterKey.getSecretKey());
}
/**
* Used by the RPC layer.
*/
@Override
public ContainerTokenIdentifier createIdentifier() {
return new ContainerTokenIdentifier();
}
/**
* Helper function for creating ContainerTokens
*
* @param containerId
* @param nodeId
* @param capability
* @return the container-token
*/
public ContainerToken createContainerToken(ContainerId containerId,
NodeId nodeId, Resource capability) {
byte[] password;
ContainerTokenIdentifier tokenIdentifier;
long expiryTimeStamp =
System.currentTimeMillis() + containerTokenExpiryInterval;
// Lock so that we use the same MasterKey's keyId and its bytes
this.readLock.lock();
try {
tokenIdentifier =
new ContainerTokenIdentifier(containerId, nodeId.toString(),
capability, expiryTimeStamp, this.currentMasterKey.getMasterKey()
.getKeyId());
password = this.createPassword(tokenIdentifier);
} finally {
this.readLock.unlock();
}
return BuilderUtils.newContainerToken(nodeId, ByteBuffer.wrap(password),
tokenIdentifier);
}
}

View File

@ -1,118 +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.security;
import java.nio.ByteBuffer;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import javax.crypto.SecretKey;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerToken;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.util.BuilderUtils;
/**
* SecretManager for ContainerTokens. Used by both RM and NM and hence is
* present in yarn-server-common package.
*
*/
public class ContainerTokenSecretManager extends
SecretManager<ContainerTokenIdentifier> {
private static Log LOG = LogFactory
.getLog(ContainerTokenSecretManager.class);
Map<String, SecretKey> secretkeys =
new ConcurrentHashMap<String, SecretKey>();
private final long containerTokenExpiryInterval;
public ContainerTokenSecretManager(Configuration conf) {
this.containerTokenExpiryInterval =
conf.getInt(YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS);
}
public ContainerToken createContainerToken(ContainerId containerId,
NodeId nodeId, Resource capability) {
try {
long expiryTimeStamp =
System.currentTimeMillis() + containerTokenExpiryInterval;
ContainerTokenIdentifier tokenIdentifier =
new ContainerTokenIdentifier(containerId, nodeId.toString(),
capability, expiryTimeStamp);
return BuilderUtils.newContainerToken(nodeId,
ByteBuffer.wrap(this.createPassword(tokenIdentifier)), tokenIdentifier);
} catch (IllegalArgumentException e) {
// this could be because DNS is down - in which case we just want
// to retry and not bring RM down. Caller should note and act on the fact
// that container is not creatable.
LOG.error("Error trying to create new container", e);
return null;
}
}
// Used by master for generation of secretyKey per host
public SecretKey createAndGetSecretKey(CharSequence hostName) {
String hostNameStr = hostName.toString();
if (!this.secretkeys.containsKey(hostNameStr)) {
LOG.debug("Creating secretKey for NM " + hostNameStr);
this.secretkeys.put(hostNameStr,
createSecretKey("mySecretKey".getBytes()));
}
return this.secretkeys.get(hostNameStr);
}
// Used by slave for using secretKey sent by the master.
public void setSecretKey(CharSequence hostName, byte[] secretKeyBytes) {
this.secretkeys.put(hostName.toString(), createSecretKey(secretKeyBytes));
}
@Override
public byte[] createPassword(ContainerTokenIdentifier identifier) {
LOG.debug("Creating password for " + identifier.getContainerID()
+ " to be run on NM " + identifier.getNmHostAddress() + " "
+ this.secretkeys.get(identifier.getNmHostAddress()));
return createPassword(identifier.getBytes(),
this.secretkeys.get(identifier.getNmHostAddress()));
}
@Override
public byte[] retrievePassword(ContainerTokenIdentifier identifier)
throws org.apache.hadoop.security.token.SecretManager.InvalidToken {
LOG.debug("Retrieving password for " + identifier.getContainerID()
+ " to be run on NM " + identifier.getNmHostAddress());
return createPassword(identifier.getBytes(),
this.secretkeys.get(identifier.getNmHostAddress()));
}
@Override
public ContainerTokenIdentifier createIdentifier() {
return new ContainerTokenIdentifier();
}
}

View File

@ -37,15 +37,21 @@ message NodeStatusProto {
repeated ApplicationIdProto keep_alive_applications = 5;
}
message MasterKeyProto {
optional int32 key_id = 1;
optional bytes bytes = 2;
}
message RegistrationResponseProto {
optional bytes secret_key = 1;
optional MasterKeyProto master_key = 1;
optional NodeActionProto nodeAction = 2;
}
message HeartbeatResponseProto {
optional int32 response_id = 1;
optional NodeActionProto nodeAction = 2;
repeated ContainerIdProto containers_to_cleanup = 3;
repeated ApplicationIdProto applications_to_cleanup = 4;
optional MasterKeyProto master_key = 2;
optional NodeActionProto nodeAction = 3;
repeated ContainerIdProto containers_to_cleanup = 4;
repeated ApplicationIdProto applications_to_cleanup = 5;
}

View File

@ -26,6 +26,7 @@
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
/**
* Context interface for sharing information across components in the
@ -44,5 +45,7 @@ public interface Context {
ConcurrentMap<ContainerId, Container> getContainers();
NMContainerTokenSecretManager getContainerTokenSecretManager();
NodeHealthStatus getNodeHealthStatus();
}

View File

@ -46,9 +46,9 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.hadoop.yarn.service.Service;
import org.apache.hadoop.yarn.service.ServiceStateChangeListener;
@ -64,7 +64,6 @@ public class NodeManager extends CompositeService implements
private static final Log LOG = LogFactory.getLog(NodeManager.class);
protected final NodeManagerMetrics metrics = NodeManagerMetrics.create();
protected ContainerTokenSecretManager containerTokenSecretManager;
private ApplicationACLsManager aclsManager;
private NodeHealthCheckerService nodeHealthChecker;
private LocalDirsHandlerService dirsHandler;
@ -75,10 +74,9 @@ public NodeManager() {
}
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
ContainerTokenSecretManager containerTokenSecretManager) {
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
return new NodeStatusUpdaterImpl(context, dispatcher, healthChecker,
metrics, containerTokenSecretManager);
metrics);
}
protected NodeResourceMonitor createNodeResourceMonitor() {
@ -87,11 +85,10 @@ protected NodeResourceMonitor createNodeResourceMonitor() {
protected ContainerManagerImpl createContainerManager(Context context,
ContainerExecutor exec, DeletionService del,
NodeStatusUpdater nodeStatusUpdater, ContainerTokenSecretManager
containerTokenSecretManager, ApplicationACLsManager aclsManager,
NodeStatusUpdater nodeStatusUpdater, ApplicationACLsManager aclsManager,
LocalDirsHandlerService dirsHandler) {
return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
metrics, containerTokenSecretManager, aclsManager, dirsHandler);
metrics, aclsManager, dirsHandler);
}
protected WebServer createWebServer(Context nmContext,
@ -110,15 +107,16 @@ public void init(Configuration conf) {
conf.setBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY, true);
Context context = new NMContext();
// Create the secretManager if need be.
NMContainerTokenSecretManager containerTokenSecretManager = null;
if (UserGroupInformation.isSecurityEnabled()) {
LOG.info("Security is enabled on NodeManager. "
+ "Creating ContainerTokenSecretManager");
this.containerTokenSecretManager = new ContainerTokenSecretManager(conf);
containerTokenSecretManager = new NMContainerTokenSecretManager(conf);
}
Context context = new NMContext(containerTokenSecretManager);
this.aclsManager = new ApplicationACLsManager(conf);
ContainerExecutor exec = ReflectionUtils.newInstance(
@ -139,8 +137,8 @@ public void init(Configuration conf) {
addService(nodeHealthChecker);
dirsHandler = nodeHealthChecker.getDiskHandler();
NodeStatusUpdater nodeStatusUpdater = createNodeStatusUpdater(context,
dispatcher, nodeHealthChecker, this.containerTokenSecretManager);
NodeStatusUpdater nodeStatusUpdater =
createNodeStatusUpdater(context, dispatcher, nodeHealthChecker);
nodeStatusUpdater.register(this);
NodeResourceMonitor nodeResourceMonitor = createNodeResourceMonitor();
@ -148,7 +146,7 @@ public void init(Configuration conf) {
ContainerManagerImpl containerManager =
createContainerManager(context, exec, del, nodeStatusUpdater,
this.containerTokenSecretManager, this.aclsManager, dirsHandler);
this.aclsManager, dirsHandler);
addService(containerManager);
Service webServer = createWebServer(context, containerManager
@ -192,10 +190,13 @@ public static class NMContext implements Context {
private final ConcurrentMap<ContainerId, Container> containers =
new ConcurrentSkipListMap<ContainerId, Container>();
private final NMContainerTokenSecretManager containerTokenSecretManager;
private final NodeHealthStatus nodeHealthStatus = RecordFactoryProvider
.getRecordFactory(null).newRecordInstance(NodeHealthStatus.class);
public NMContext() {
public NMContext(NMContainerTokenSecretManager containerTokenSecretManager) {
this.containerTokenSecretManager = containerTokenSecretManager;
this.nodeHealthStatus.setIsNodeHealthy(true);
this.nodeHealthStatus.setHealthReport("Healthy");
this.nodeHealthStatus.setLastHealthReportTime(System.currentTimeMillis());
@ -219,6 +220,10 @@ public ConcurrentMap<ContainerId, Container> getContainers() {
return this.containers;
}
@Override
public NMContainerTokenSecretManager getContainerTokenSecretManager() {
return this.containerTokenSecretManager;
}
@Override
public NodeHealthStatus getNodeHealthStatus() {
return this.nodeHealthStatus;

View File

@ -22,7 +22,5 @@
public interface NodeStatusUpdater extends Service {
byte[] getRMNMSharedSecret();
void sendOutofBandHeartBeat();
}

View File

@ -25,8 +25,8 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Map.Entry;
import java.util.Random;
import org.apache.avro.AvroRuntimeException;
import org.apache.commons.logging.Log;
@ -51,15 +51,14 @@
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeAction;
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.service.AbstractService;
public class NodeStatusUpdaterImpl extends AbstractService implements
NodeStatusUpdater {
@ -71,13 +70,11 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
private final Dispatcher dispatcher;
private NodeId nodeId;
private ContainerTokenSecretManager containerTokenSecretManager;
private long heartBeatInterval;
private ResourceTracker resourceTracker;
private InetSocketAddress rmAddress;
private Resource totalResource;
private int httpPort;
private byte[] secretKeyBytes = new byte[0];
private boolean isStopped;
private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
private boolean tokenKeepAliveEnabled;
@ -93,14 +90,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
private boolean hasToRebootNode;
public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics,
ContainerTokenSecretManager containerTokenSecretManager) {
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
super(NodeStatusUpdaterImpl.class.getName());
this.healthChecker = healthChecker;
this.context = context;
this.dispatcher = dispatcher;
this.metrics = metrics;
this.containerTokenSecretManager = containerTokenSecretManager;
}
@Override
@ -196,28 +191,22 @@ private void registerWithRM() throws YarnRemoteException {
}
if (UserGroupInformation.isSecurityEnabled()) {
this.secretKeyBytes = regResponse.getSecretKey().array();
}
MasterKey masterKey = regResponse.getMasterKey();
// do this now so that its set before we start heartbeating to RM
if (UserGroupInformation.isSecurityEnabled()) {
LOG.info("Security enabled - updating secret keys now");
// It is expected that status updater is started by this point and
// RM gives the shared secret in registration during StatusUpdater#start().
this.containerTokenSecretManager.setSecretKey(
this.nodeId.toString(),
this.getRMNMSharedSecret());
// RM gives the shared secret in registration during
// StatusUpdater#start().
if (masterKey != null) {
this.context.getContainerTokenSecretManager().setMasterKey(masterKey);
}
}
LOG.info("Registered with ResourceManager as " + this.nodeId
+ " with total resource of " + this.totalResource);
}
@Override
public byte[] getRMNMSharedSecret() {
return this.secretKeyBytes.clone();
}
private List<ApplicationId> createKeepAliveApplicationList() {
if (!tokenKeepAliveEnabled) {
return Collections.emptyList();
@ -335,6 +324,17 @@ public void run() {
request.setNodeStatus(nodeStatus);
HeartbeatResponse response =
resourceTracker.nodeHeartbeat(request).getHeartbeatResponse();
// See if the master-key has rolled over
if (isSecurityEnabled()) {
MasterKey updatedMasterKey = response.getMasterKey();
if (updatedMasterKey != null) {
// Will be non-null only on roll-over on RM side
context.getContainerTokenSecretManager().setMasterKey(
updatedMasterKey);
}
}
if (response.getNodeAction() == NodeAction.SHUTDOWN) {
LOG
.info("Recieved SHUTDOWN signal from Resourcemanager as part of heartbeat," +

View File

@ -21,10 +21,10 @@
import static org.apache.hadoop.yarn.service.Service.STATE.STARTED;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.util.Map;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -96,7 +96,6 @@
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.nodemanager.security.authorize.NMPolicyProvider;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.hadoop.yarn.service.Service;
import org.apache.hadoop.yarn.service.ServiceStateChangeListener;
@ -110,14 +109,12 @@ public class ContainerManagerImpl extends CompositeService implements
final Context context;
private final ContainersMonitor containersMonitor;
private Server server;
private InetAddress resolvedAddress = null;
private final ResourceLocalizationService rsrcLocalizationSrvc;
private final ContainersLauncher containersLauncher;
private final AuxServices auxiliaryServices;
private final NodeManagerMetrics metrics;
private final NodeStatusUpdater nodeStatusUpdater;
private ContainerTokenSecretManager containerTokenSecretManager;
private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
@ -129,8 +126,7 @@ public class ContainerManagerImpl extends CompositeService implements
public ContainerManagerImpl(Context context, ContainerExecutor exec,
DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater,
NodeManagerMetrics metrics, ContainerTokenSecretManager
containerTokenSecretManager, ApplicationACLsManager aclsManager,
NodeManagerMetrics metrics, ApplicationACLsManager aclsManager,
LocalDirsHandlerService dirsHandler) {
super(ContainerManagerImpl.class.getName());
this.context = context;
@ -149,7 +145,6 @@ public ContainerManagerImpl(Context context, ContainerExecutor exec,
addService(containersLauncher);
this.nodeStatusUpdater = nodeStatusUpdater;
this.containerTokenSecretManager = containerTokenSecretManager;
this.aclsManager = aclsManager;
// Start configurable services
@ -232,7 +227,7 @@ public void start() {
server =
rpc.getServer(ContainerManager.class, this, initialAddress, conf,
this.containerTokenSecretManager,
this.context.getContainerTokenSecretManager(),
conf.getInt(YarnConfiguration.NM_CONTAINER_MGR_THREAD_COUNT,
YarnConfiguration.DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT));
@ -267,24 +262,9 @@ public void stop() {
super.stop();
}
/**
* Authorize the request.
*
* @param containerID
* of the container
* @param launchContext
* passed if verifying the startContainer, null otherwise.
* @throws YarnRemoteException
*/
private void authorizeRequest(ContainerId containerID,
ContainerLaunchContext launchContext) throws YarnRemoteException {
if (!UserGroupInformation.isSecurityEnabled()) {
return;
}
String containerIDStr = containerID.toString();
// Get the remoteUGI corresponding to the api call.
private UserGroupInformation getRemoteUgi(String containerIDStr)
throws YarnRemoteException {
UserGroupInformation remoteUgi;
try {
remoteUgi = UserGroupInformation.getCurrentUser();
@ -295,28 +275,65 @@ private void authorizeRequest(ContainerId containerID,
LOG.warn(msg);
throw RPCUtil.getRemoteException(msg);
}
return remoteUgi;
}
// Obtain the needed ContainerTokenIdentifier from the remote-UGI. RPC layer
// currently sets only the required id, but iterate through anyways just to
// be sure.
private ContainerTokenIdentifier selectContainerTokenIdentifier(
UserGroupInformation remoteUgi) {
Set<TokenIdentifier> tokenIdentifiers = remoteUgi.getTokenIdentifiers();
ContainerTokenIdentifier resultId = null;
for (TokenIdentifier id : tokenIdentifiers) {
if (id instanceof ContainerTokenIdentifier) {
resultId = (ContainerTokenIdentifier) id;
break;
}
}
return resultId;
}
/**
* Authorize the request.
*
* @param containerIDStr
* of the container
* @param launchContext
* passed if verifying the startContainer, null otherwise.
* @param remoteUgi
* ugi corresponding to the remote end making the api-call
* @throws YarnRemoteException
*/
private void authorizeRequest(String containerIDStr,
ContainerLaunchContext launchContext, UserGroupInformation remoteUgi)
throws YarnRemoteException {
if (!UserGroupInformation.isSecurityEnabled()) {
return;
}
boolean unauthorized = false;
StringBuilder messageBuilder = new StringBuilder(
"Unauthorized request to start container. ");
StringBuilder messageBuilder =
new StringBuilder("Unauthorized request to start container. ");
if (!remoteUgi.getUserName().equals(containerIDStr)) {
unauthorized = true;
messageBuilder.append("\nExpected containerId: "
+ remoteUgi.getUserName() + " Found: " + containerIDStr);
}
if (launchContext != null) {
// Verify other things for startContainer() request.
} else if (launchContext != null) {
// Verify other things also for startContainer() request.
if (LOG.isDebugEnabled()) {
LOG.debug("Number of TokenIdentifiers in the UGI from RPC: "
+ remoteUgi.getTokenIdentifiers().size());
}
// We must and should get only one TokenIdentifier from the RPC.
ContainerTokenIdentifier tokenId = (ContainerTokenIdentifier) remoteUgi
.getTokenIdentifiers().iterator().next();
// Get the tokenId from the remote user ugi
ContainerTokenIdentifier tokenId =
selectContainerTokenIdentifier(remoteUgi);
if (tokenId == null) {
unauthorized = true;
messageBuilder
@ -324,6 +341,15 @@ private void authorizeRequest(ContainerId containerID,
+ containerIDStr);
} else {
// Is the container being relaunched? Or RPC layer let startCall with
// tokens generated off old-secret through
if (!this.context.getContainerTokenSecretManager()
.isValidStartContainerRequest(tokenId)) {
unauthorized = true;
messageBuilder.append("\n Attempt to relaunch the same " +
"container with id " + containerIDStr + ".");
}
// Ensure the token is not expired.
// Token expiry is not checked for stopContainer/getContainerStatus
if (tokenId.getExpiryTimeStamp() < System.currentTimeMillis()) {
@ -359,10 +385,13 @@ public StartContainerResponse startContainer(StartContainerRequest request)
ContainerLaunchContext launchContext = request.getContainerLaunchContext();
ContainerId containerID = launchContext.getContainerId();
authorizeRequest(containerID, launchContext);
String containerIDStr = containerID.toString();
LOG.info("Start request for " + launchContext.getContainerId()
+ " by user " + launchContext.getUser());
UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr);
authorizeRequest(containerIDStr, launchContext, remoteUgi);
LOG.info("Start request for " + containerIDStr + " by user "
+ launchContext.getUser());
// //////////// Parse credentials
ByteBuffer tokens = launchContext.getContainerTokens();
@ -394,7 +423,7 @@ public StartContainerResponse startContainer(StartContainerRequest request)
AuditConstants.START_CONTAINER, "ContainerManagerImpl",
"Container already running on this node!",
applicationID, containerID);
throw RPCUtil.getRemoteException("Container " + containerID
throw RPCUtil.getRemoteException("Container " + containerIDStr
+ " already is running on this node!!");
}
@ -414,6 +443,12 @@ public StartContainerResponse startContainer(StartContainerRequest request)
// TODO: Validate the request
dispatcher.getEventHandler().handle(
new ApplicationContainerInitEvent(container));
if (UserGroupInformation.isSecurityEnabled()) {
ContainerTokenIdentifier tokenId =
selectContainerTokenIdentifier(remoteUgi);
this.context.getContainerTokenSecretManager().startContainerSuccessful(
tokenId);
}
NMAuditLogger.logSuccess(launchContext.getUser(),
AuditConstants.START_CONTAINER, "ContainerManageImpl",
@ -438,8 +473,12 @@ public StopContainerResponse stopContainer(StopContainerRequest request)
throws YarnRemoteException {
ContainerId containerID = request.getContainerId();
String containerIDStr = containerID.toString();
// TODO: Only the container's owner can kill containers today.
authorizeRequest(containerID, null);
UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr);
authorizeRequest(containerIDStr, null, remoteUgi);
StopContainerResponse response =
recordFactory.newRecordInstance(StopContainerResponse.class);
@ -476,10 +515,14 @@ public GetContainerStatusResponse getContainerStatus(
GetContainerStatusRequest request) throws YarnRemoteException {
ContainerId containerID = request.getContainerId();
// TODO: Only the container's owner can get containers' status today.
authorizeRequest(containerID, null);
String containerIDStr = containerID.toString();
LOG.info("Getting container-status for " + containerID);
// TODO: Only the container's owner can get containers' status today.
UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr);
authorizeRequest(containerIDStr, null, remoteUgi);
LOG.info("Getting container-status for " + containerIDStr);
Container container = this.context.getContainers().get(containerID);
if (container != null) {
ContainerStatus containerStatus = container.cloneAndGetContainerStatus();
@ -490,7 +533,7 @@ public GetContainerStatusResponse getContainerStatus(
return response;
}
throw RPCUtil.getRemoteException("Container " + containerID
throw RPCUtil.getRemoteException("Container " + containerIDStr
+ " is not handled by this NodeManager");
}

View File

@ -28,8 +28,9 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy;
@ -42,6 +43,7 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ApplicationLocalizationEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppStartedEvent;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@ -365,6 +367,10 @@ static class AppCompletelyDoneTransition implements
@Override
public void transition(ApplicationImpl app, ApplicationEvent event) {
// Inform the ContainerTokenSecretManager
if (UserGroupInformation.isSecurityEnabled()) {
app.context.getContainerTokenSecretManager().appFinished(app.appId);
}
// Inform the logService
app.dispatcher.getEventHandler().handle(
new LogHandlerAppFinishedEvent(app.appId));

View File

@ -0,0 +1,189 @@
/**
* 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.nodemanager.security;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
/**
* The NM maintains only two master-keys. The current key that RM knows and the
* key from the previous rolling-interval.
*
*/
public class NMContainerTokenSecretManager extends
BaseContainerTokenSecretManager {
private static final Log LOG = LogFactory
.getLog(NMContainerTokenSecretManager.class);
private MasterKeyData previousMasterKey;
private final Map<ApplicationId, ConcurrentMap<ContainerId, MasterKeyData>> oldMasterKeys;
public NMContainerTokenSecretManager(Configuration conf) {
super(conf);
this.oldMasterKeys =
new HashMap<ApplicationId, ConcurrentMap<ContainerId, MasterKeyData>>();
}
/**
* Used by NodeManagers to create a token-secret-manager with the key obtained
* from the RM. This can happen during registration or when the RM rolls the
* master-key and signals the NM.
*
* @param masterKeyRecord
*/
@Private
public synchronized void setMasterKey(MasterKey masterKeyRecord) {
LOG.info("Rolling master-key for container-tokens, got key with id "
+ masterKeyRecord.getKeyId());
if (super.currentMasterKey == null) {
super.currentMasterKey = new MasterKeyData(masterKeyRecord);
} else {
if (super.currentMasterKey.getMasterKey().getKeyId() != masterKeyRecord
.getKeyId()) {
// Update keys only if the key has changed.
this.previousMasterKey = super.currentMasterKey;
super.currentMasterKey = new MasterKeyData(masterKeyRecord);
}
}
}
/**
* Override of this is to validate ContainerTokens generated by using
* different {@link MasterKey}s.
*/
@Override
public synchronized byte[] retrievePassword(
ContainerTokenIdentifier identifier) throws SecretManager.InvalidToken {
int keyId = identifier.getMasterKeyId();
ContainerId containerId = identifier.getContainerID();
ApplicationId appId =
containerId.getApplicationAttemptId().getApplicationId();
MasterKeyData masterKeyToUse = null;
if (this.previousMasterKey != null
&& keyId == this.previousMasterKey.getMasterKey().getKeyId()) {
// A container-launch has come in with a token generated off the last
// master-key
masterKeyToUse = this.previousMasterKey;
} else if (keyId == super.currentMasterKey.getMasterKey().getKeyId()) {
// A container-launch has come in with a token generated off the current
// master-key
masterKeyToUse = super.currentMasterKey;
} else if (this.oldMasterKeys.containsKey(appId)
&& this.oldMasterKeys.get(appId).containsKey(containerId)) {
// This means on the following happened:
// (1) a stopContainer() or a getStatus() happened for a container with
// token generated off a master-key that is neither current nor the
// previous one.
// (2) a container-relaunch has come in with a token generated off a
// master-key that is neither current nor the previous one.
// This basically lets stop and getStatus() calls with old-tokens to pass
// through without any issue, i.e. (1).
// Start-calls for repetitive launches (2) also pass through RPC here, but
// get thwarted at the app-layer as part of startContainer() call.
masterKeyToUse = this.oldMasterKeys.get(appId).get(containerId);
}
if (masterKeyToUse != null) {
return retrievePasswordInternal(identifier, masterKeyToUse);
}
// Invalid request. Like startContainer() with token generated off
// old-master-keys.
throw new SecretManager.InvalidToken("Given Container "
+ identifier.getContainerID().toString()
+ " seems to have an illegally generated token.");
}
/**
* Container start has gone through. Store the corresponding keys so that
* stopContainer() and getContainerStatus() can be authenticated long after
* the container-start went through.
*/
public synchronized void startContainerSuccessful(
ContainerTokenIdentifier tokenId) {
if (!UserGroupInformation.isSecurityEnabled()) {
return;
}
int keyId = tokenId.getMasterKeyId();
if (currentMasterKey.getMasterKey().getKeyId() == keyId) {
addKeyForContainerId(tokenId.getContainerID(), currentMasterKey);
} else if (previousMasterKey != null
&& previousMasterKey.getMasterKey().getKeyId() == keyId) {
addKeyForContainerId(tokenId.getContainerID(), previousMasterKey);
}
}
/**
* Ensure the startContainer call is not using an older cached key. Will
* return false once startContainerSuccessful is called. Does not check
* the actual key being current since that is verified by the security layer
* via retrievePassword.
*/
public synchronized boolean isValidStartContainerRequest(
ContainerTokenIdentifier tokenId) {
ContainerId containerID = tokenId.getContainerID();
ApplicationId applicationId =
containerID.getApplicationAttemptId().getApplicationId();
return !this.oldMasterKeys.containsKey(applicationId)
|| !this.oldMasterKeys.get(applicationId).containsKey(containerID);
}
private synchronized void addKeyForContainerId(ContainerId containerId,
MasterKeyData masterKeyData) {
if (containerId != null) {
ApplicationId appId =
containerId.getApplicationAttemptId().getApplicationId();
if (!this.oldMasterKeys.containsKey(appId)) {
this.oldMasterKeys.put(appId,
new ConcurrentHashMap<ContainerId, MasterKeyData>());
}
ConcurrentMap<ContainerId, MasterKeyData> containerIdToKeysMapForThisApp =
this.oldMasterKeys.get(appId);
containerIdToKeysMapForThisApp.put(containerId, masterKeyData);
} else {
LOG.warn("Not adding key for null containerId");
}
}
// Holding on to master-keys corresponding to containers until the app is
// finished due to the multiple ways a container can finish. Avoid
// stopContainer calls seeing unnecessary authorization exceptions.
public synchronized void appFinished(ApplicationId appId) {
this.oldMasterKeys.remove(appId);
}
}

View File

@ -27,8 +27,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
@ -50,6 +48,7 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEvent;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
public class DummyContainerManager extends ContainerManagerImpl {
@ -59,11 +58,10 @@ public class DummyContainerManager extends ContainerManagerImpl {
public DummyContainerManager(Context context, ContainerExecutor exec,
DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater,
NodeManagerMetrics metrics,
ContainerTokenSecretManager containerTokenSecretManager,
ApplicationACLsManager applicationACLsManager,
LocalDirsHandlerService dirsHandler) {
super(context, exec, deletionContext, nodeStatusUpdater, metrics,
containerTokenSecretManager, applicationACLsManager, dirsHandler);
applicationACLsManager, dirsHandler);
}
@Override

View File

@ -36,12 +36,12 @@
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.api.ResourceTracker;
import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.junit.Test;
public class TestEventFlow {
@ -69,9 +69,9 @@ public void testSuccessfulContainerLaunch() throws InterruptedException,
localLogDir.mkdir();
remoteLogDir.mkdir();
Context context = new NMContext();
YarnConfiguration conf = new YarnConfiguration();
Context context = new NMContext(new NMContainerTokenSecretManager(conf));
conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
@ -86,10 +86,8 @@ public void testSuccessfulContainerLaunch() throws InterruptedException,
healthChecker.init(conf);
LocalDirsHandlerService dirsHandler = healthChecker.getDiskHandler();
NodeManagerMetrics metrics = NodeManagerMetrics.create();
ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager(conf);
NodeStatusUpdater nodeStatusUpdater =
new NodeStatusUpdaterImpl(context, dispatcher, healthChecker, metrics, containerTokenSecretManager) {
new NodeStatusUpdaterImpl(context, dispatcher, healthChecker, metrics) {
@Override
protected ResourceTracker getRMClient() {
return new LocalRMInterface();
@ -101,10 +99,9 @@ protected void startStatusUpdater() {
}
};
DummyContainerManager containerManager = new DummyContainerManager(
context, exec, del, nodeStatusUpdater, metrics,
containerTokenSecretManager, new ApplicationACLsManager(conf),
dirsHandler);
DummyContainerManager containerManager =
new DummyContainerManager(context, exec, del, nodeStatusUpdater,
metrics, new ApplicationACLsManager(conf), dirsHandler);
containerManager.init(conf);
containerManager.start();

View File

@ -17,37 +17,29 @@
*/
package org.apache.hadoop.yarn.server.nodemanager;
import static junit.framework.Assert.assertEquals;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.ipc.TestRPC.TestImpl;
import org.apache.hadoop.ipc.TestRPC.TestProtocol;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger;
import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.AuditConstants;
import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.Keys;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.net.NetUtils;
import static org.mockito.Mockito.*;
import static junit.framework.Assert.*;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
/**
* Tests {@link NMAuditLogger}.
*/
public class TestNMAuditLogger {
private static final Log LOG = LogFactory.getLog(TestNMAuditLogger.class);
private static final String USER = "test";
private static final String OPERATION = "oper";
private static final String TARGET = "tgt";

View File

@ -65,7 +65,6 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.service.Service;
import org.apache.hadoop.yarn.service.Service.STATE;
import org.apache.hadoop.yarn.util.BuilderUtils;
@ -234,10 +233,8 @@ private class MyNodeStatusUpdater extends NodeStatusUpdaterImpl {
private Context context;
public MyNodeStatusUpdater(Context context, Dispatcher dispatcher,
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics,
ContainerTokenSecretManager containerTokenSecretManager) {
super(context, dispatcher, healthChecker, metrics,
containerTokenSecretManager);
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
super(context, dispatcher, healthChecker, metrics);
this.context = context;
}
@ -252,10 +249,8 @@ private class MyNodeStatusUpdater3 extends NodeStatusUpdaterImpl {
private Context context;
public MyNodeStatusUpdater3(Context context, Dispatcher dispatcher,
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics,
ContainerTokenSecretManager containerTokenSecretManager) {
super(context, dispatcher, healthChecker, metrics,
containerTokenSecretManager);
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
super(context, dispatcher, healthChecker, metrics);
this.context = context;
this.resourceTracker = new MyResourceTracker3(this.context);
}
@ -276,11 +271,9 @@ private class MyNodeManager extends NodeManager {
private MyNodeStatusUpdater3 nodeStatusUpdater;
@Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
ContainerTokenSecretManager containerTokenSecretManager) {
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
this.nodeStatusUpdater =
new MyNodeStatusUpdater3(context, dispatcher, healthChecker, metrics,
containerTokenSecretManager);
new MyNodeStatusUpdater3(context, dispatcher, healthChecker, metrics);
return this.nodeStatusUpdater;
}
@ -398,10 +391,9 @@ public void testNMRegistration() throws InterruptedException {
nm = new NodeManager() {
@Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
ContainerTokenSecretManager containerTokenSecretManager) {
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
return new MyNodeStatusUpdater(context, dispatcher, healthChecker,
metrics, containerTokenSecretManager);
metrics);
}
};
@ -528,11 +520,9 @@ public void testNMShutdownForRegistrationFailure() {
nm = new NodeManager() {
@Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
ContainerTokenSecretManager containerTokenSecretManager) {
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
MyNodeStatusUpdater nodeStatusUpdater = new MyNodeStatusUpdater(
context, dispatcher, healthChecker, metrics,
containerTokenSecretManager);
context, dispatcher, healthChecker, metrics);
MyResourceTracker2 myResourceTracker2 = new MyResourceTracker2();
myResourceTracker2.registerNodeAction = NodeAction.SHUTDOWN;
nodeStatusUpdater.resourceTracker = myResourceTracker2;
@ -556,22 +546,19 @@ public void testNoRegistrationWhenNMServicesFail() {
nm = new NodeManager() {
@Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
ContainerTokenSecretManager containerTokenSecretManager) {
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
return new MyNodeStatusUpdater(context, dispatcher, healthChecker,
metrics, containerTokenSecretManager);
metrics);
}
@Override
protected ContainerManagerImpl createContainerManager(Context context,
ContainerExecutor exec, DeletionService del,
NodeStatusUpdater nodeStatusUpdater,
ContainerTokenSecretManager containerTokenSecretManager,
ApplicationACLsManager aclsManager,
LocalDirsHandlerService diskhandler) {
return new ContainerManagerImpl(context, exec, del,
nodeStatusUpdater, metrics, containerTokenSecretManager,
aclsManager, diskhandler) {
return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
metrics, aclsManager, diskhandler) {
@Override
public void start() {
// Simulating failure of starting RPC server
@ -654,11 +641,9 @@ private NodeManager getNodeManager(final NodeAction nodeHeartBeatAction) {
return new NodeManager() {
@Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
ContainerTokenSecretManager containerTokenSecretManager) {
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
MyNodeStatusUpdater myNodeStatusUpdater = new MyNodeStatusUpdater(
context, dispatcher, healthChecker, metrics,
containerTokenSecretManager);
context, dispatcher, healthChecker, metrics);
MyResourceTracker2 myResourceTracker2 = new MyResourceTracker2();
myResourceTracker2.heartBeatNodeAction = nodeHeartBeatAction;
myNodeStatusUpdater.resourceTracker = myResourceTracker2;

View File

@ -54,8 +54,8 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.service.Service.STATE;
import org.junit.After;
import org.junit.Before;
@ -70,8 +70,6 @@ public abstract class BaseContainerManagerTest {
protected static File localLogDir;
protected static File remoteLogDir;
protected static File tmpDir;
protected ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager(new Configuration());
protected final NodeManagerMetrics metrics = NodeManagerMetrics.create();
@ -93,7 +91,8 @@ public BaseContainerManagerTest() throws UnsupportedFileSystemException {
.getLog(BaseContainerManagerTest.class);
protected Configuration conf = new YarnConfiguration();
protected Context context = new NMContext();
protected Context context = new NMContext(new NMContainerTokenSecretManager(
conf));
protected ContainerExecutor exec;
protected DeletionService delSrvc;
protected String user = "nobody";
@ -101,7 +100,7 @@ public BaseContainerManagerTest() throws UnsupportedFileSystemException {
protected LocalDirsHandlerService dirsHandler;
protected NodeStatusUpdater nodeStatusUpdater = new NodeStatusUpdaterImpl(
context, new AsyncDispatcher(), null, metrics, this.containerTokenSecretManager) {
context, new AsyncDispatcher(), null, metrics) {
@Override
protected ResourceTracker getRMClient() {
return new LocalRMInterface();
@ -155,9 +154,9 @@ public void delete(String user, Path subDir, Path[] baseDirs) {
nodeHealthChecker = new NodeHealthCheckerService();
nodeHealthChecker.init(conf);
dirsHandler = nodeHealthChecker.getDiskHandler();
containerManager = new ContainerManagerImpl(context, exec, delSrvc,
nodeStatusUpdater, metrics, this.containerTokenSecretManager,
new ApplicationACLsManager(conf), dirsHandler);
containerManager =
new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
metrics, new ApplicationACLsManager(conf), dirsHandler);
containerManager.init(conf);
}

View File

@ -51,14 +51,13 @@
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.junit.Test;
@ -384,11 +383,9 @@ public void testLocalFilesCleanup() throws InterruptedException,
delSrvc = new DeletionService(exec);
delSrvc.init(conf);
ContainerTokenSecretManager containerTokenSecretManager = new
ContainerTokenSecretManager(conf);
containerManager = new ContainerManagerImpl(context, exec, delSrvc,
nodeStatusUpdater, metrics, containerTokenSecretManager,
new ApplicationACLsManager(conf), dirsHandler);
containerManager =
new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
metrics, new ApplicationACLsManager(conf), dirsHandler);
containerManager.init(conf);
containerManager.start();

View File

@ -30,9 +30,9 @@
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.event.DrainDispatcher;
@ -50,8 +50,8 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ApplicationLocalizationEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEventType;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEventType;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;

View File

@ -76,7 +76,7 @@ public void tearDown() {
}
private String startNMWebAppServer(String webAddr) {
Context nmContext = new NodeManager.NMContext();
Context nmContext = new NodeManager.NMContext(null);
ResourceView resourceView = new ResourceView() {
@Override
public long getVmemAllocatedForContainers() {
@ -116,7 +116,7 @@ public void testNMWebAppWithEphemeralPort() throws IOException {
@Test
public void testNMWebApp() throws IOException {
Context nmContext = new NodeManager.NMContext();
Context nmContext = new NodeManager.NMContext(null);
ResourceView resourceView = new ResourceView() {
@Override
public long getVmemAllocatedForContainers() {

View File

@ -85,7 +85,7 @@ public class TestNMWebServices extends JerseyTest {
private Injector injector = Guice.createInjector(new ServletModule() {
@Override
protected void configureServlets() {
nmContext = new NodeManager.NMContext();
nmContext = new NodeManager.NMContext(null);
nmContext.getNodeId().setHost("testhost.foo.com");
nmContext.getNodeId().setPort(8042);
resourceView = new ResourceView() {

View File

@ -91,7 +91,7 @@ public class TestNMWebServicesApps extends JerseyTest {
private Injector injector = Guice.createInjector(new ServletModule() {
@Override
protected void configureServlets() {
nmContext = new NodeManager.NMContext();
nmContext = new NodeManager.NMContext(null);
nmContext.getNodeId().setHost("testhost.foo.com");
nmContext.getNodeId().setPort(9999);
resourceView = new ResourceView() {

View File

@ -91,7 +91,7 @@ public class TestNMWebServicesContainers extends JerseyTest {
private Injector injector = Guice.createInjector(new ServletModule() {
@Override
protected void configureServlets() {
nmContext = new NodeManager.NMContext();
nmContext = new NodeManager.NMContext(null);
nmContext.getNodeId().setHost("testhost.foo.com");
nmContext.getNodeId().setPort(8042);
resourceView = new ResourceView() {

View File

@ -21,11 +21,11 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.ipc.Server;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.Groups;
import org.apache.hadoop.security.UserGroupInformation;
@ -171,8 +171,7 @@ public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
throws YarnRemoteException {
UserGroupInformation user = checkAcls("refreshQueues");
try {
scheduler.reinitialize(conf, null, null); // ContainerTokenSecretManager can't
// be 'refreshed'
scheduler.reinitialize(conf, this.rmContext);
RMAuditLogger.logSuccess(user.getShortUserName(), "refreshQueues",
"AdminService");
return recordFactory.newRecordInstance(RefreshQueuesResponse.class);

View File

@ -31,6 +31,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
/**
* Context of the ResourceManager.
@ -58,4 +59,6 @@ public interface RMContext {
DelegationTokenRenewer getDelegationTokenRenewer();
ApplicationTokenSecretManager getApplicationTokenSecretManager();
RMContainerTokenSecretManager getContainerTokenSecretManager();
}

View File

@ -33,6 +33,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
public class RMContextImpl implements RMContext {
@ -53,13 +54,15 @@ public class RMContextImpl implements RMContext {
private ContainerAllocationExpirer containerAllocationExpirer;
private final DelegationTokenRenewer tokenRenewer;
private final ApplicationTokenSecretManager appTokenSecretManager;
private final RMContainerTokenSecretManager containerTokenSecretManager;
public RMContextImpl(Store store, Dispatcher rmDispatcher,
ContainerAllocationExpirer containerAllocationExpirer,
AMLivelinessMonitor amLivelinessMonitor,
AMLivelinessMonitor amFinishingMonitor,
DelegationTokenRenewer tokenRenewer,
ApplicationTokenSecretManager appTokenSecretManager) {
ApplicationTokenSecretManager appTokenSecretManager,
RMContainerTokenSecretManager containerTokenSecretManager) {
this.store = store;
this.rmDispatcher = rmDispatcher;
this.containerAllocationExpirer = containerAllocationExpirer;
@ -67,6 +70,7 @@ public RMContextImpl(Store store, Dispatcher rmDispatcher,
this.amFinishingMonitor = amFinishingMonitor;
this.tokenRenewer = tokenRenewer;
this.appTokenSecretManager = appTokenSecretManager;
this.containerTokenSecretManager = containerTokenSecretManager;
}
@Override
@ -123,4 +127,9 @@ public DelegationTokenRenewer getDelegationTokenRenewer() {
public ApplicationTokenSecretManager getApplicationTokenSecretManager() {
return this.appTokenSecretManager;
}
@Override
public RMContainerTokenSecretManager getContainerTokenSecretManager() {
return this.containerTokenSecretManager;
}
}

View File

@ -48,8 +48,8 @@
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@ -67,9 +67,9 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.webproxy.AppReportFetcher;
import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
import org.apache.hadoop.yarn.server.webproxy.WebAppProxy;
@ -100,7 +100,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
protected ClientToAMSecretManager clientToAMSecretManager =
new ClientToAMSecretManager();
protected ContainerTokenSecretManager containerTokenSecretManager;
protected RMContainerTokenSecretManager containerTokenSecretManager;
protected ApplicationTokenSecretManager appTokenSecretManager;
@ -150,8 +150,6 @@ public synchronized void init(Configuration conf) {
this.rmDispatcher);
addService(this.containerAllocationExpirer);
this.containerTokenSecretManager = new ContainerTokenSecretManager(conf);
AMLivelinessMonitor amLivelinessMonitor = createAMLivelinessMonitor();
addService(amLivelinessMonitor);
@ -161,10 +159,13 @@ public synchronized void init(Configuration conf) {
DelegationTokenRenewer tokenRenewer = createDelegationTokenRenewer();
addService(tokenRenewer);
this.rmContext = new RMContextImpl(this.store, this.rmDispatcher,
this.containerAllocationExpirer,
amLivelinessMonitor, amFinishingMonitor,
tokenRenewer, this.appTokenSecretManager);
this.containerTokenSecretManager = new RMContainerTokenSecretManager(conf);
this.rmContext =
new RMContextImpl(this.store, this.rmDispatcher,
this.containerAllocationExpirer, amLivelinessMonitor,
amFinishingMonitor, tokenRenewer, this.appTokenSecretManager,
this.containerTokenSecretManager);
// Register event handler for NodesListManager
this.nodesListManager = new NodesListManager(this.rmContext);
@ -198,8 +199,7 @@ public synchronized void init(Configuration conf) {
addService(resourceTracker);
try {
this.scheduler.reinitialize(conf,
this.containerTokenSecretManager, this.rmContext);
this.scheduler.reinitialize(conf, this.rmContext);
} catch (IOException ioe) {
throw new RuntimeException("Failed to initialize scheduler", ioe);
}
@ -486,6 +486,7 @@ public void start() {
}
this.appTokenSecretManager.start();
this.containerTokenSecretManager.start();
startWepApp();
DefaultMetricsSystem.initialize("ResourceManager");
@ -531,6 +532,7 @@ public void stop() {
rmDTSecretManager.stopThreads();
this.appTokenSecretManager.stop();
this.containerTokenSecretManager.stop();
/*synchronized(shutdown) {
shutdown.set(true);
@ -616,7 +618,7 @@ public ApplicationACLsManager getApplicationACLsManager() {
}
@Private
public ContainerTokenSecretManager getContainerTokenSecretManager() {
public RMContainerTokenSecretManager getRMContainerTokenSecretManager() {
return this.containerTokenSecretManager;
}

View File

@ -18,9 +18,6 @@
package org.apache.hadoop.yarn.server.resourcemanager;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import javax.crypto.SecretKey;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -28,6 +25,7 @@
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
@ -42,6 +40,7 @@
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeAction;
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
@ -52,8 +51,8 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeReconnectEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.util.RackResolver;
@ -68,7 +67,7 @@ public class ResourceTrackerService extends AbstractService implements
private final RMContext rmContext;
private final NodesListManager nodesListManager;
private final NMLivelinessMonitor nmLivelinessMonitor;
private final ContainerTokenSecretManager containerTokenSecretManager;
private final RMContainerTokenSecretManager containerTokenSecretManager;
private Server server;
private InetSocketAddress resourceTrackerAddress;
@ -93,7 +92,7 @@ public class ResourceTrackerService extends AbstractService implements
public ResourceTrackerService(RMContext rmContext,
NodesListManager nodesListManager,
NMLivelinessMonitor nmLivelinessMonitor,
ContainerTokenSecretManager containerTokenSecretManager) {
RMContainerTokenSecretManager containerTokenSecretManager) {
super(ResourceTrackerService.class.getName());
this.rmContext = rmContext;
this.nodesListManager = nodesListManager;
@ -160,9 +159,6 @@ public RegisterNodeManagerResponse registerNodeManager(
.newRecordInstance(RegisterNodeManagerResponse.class);
RegistrationResponse regResponse = recordFactory
.newRecordInstance(RegistrationResponse.class);
SecretKey secretKey = this.containerTokenSecretManager
.createAndGetSecretKey(nodeId.toString());
regResponse.setSecretKey(ByteBuffer.wrap(secretKey.getEncoded()));
// Check if this node is a 'valid' node
if (!this.nodesListManager.isValidNode(host)) {
@ -173,8 +169,14 @@ public RegisterNodeManagerResponse registerNodeManager(
return response;
}
MasterKey nextMasterKeyForNode = null;
if (isSecurityEnabled()) {
nextMasterKeyForNode = this.containerTokenSecretManager.getCurrentKey();
regResponse.setMasterKey(nextMasterKeyForNode);
}
RMNode rmNode = new RMNodeImpl(nodeId, rmContext, host, cmPort, httpPort,
resolve(host), capability);
resolve(host), capability, nextMasterKeyForNode);
RMNode oldNode = this.rmContext.getRMNodes().putIfAbsent(nodeId, rmNode);
if (oldNode == null) {
@ -264,11 +266,32 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
latestResponse.addAllApplicationsToCleanup(rmNode.getAppsToCleanup());
latestResponse.setNodeAction(NodeAction.NORMAL);
MasterKey nextMasterKeyForNode = null;
// Check if node's masterKey needs to be updated and if the currentKey has
// roller over, send it across
if (isSecurityEnabled()) {
boolean shouldSendMasterKey = false;
MasterKey nodeKnownMasterKey = rmNode.getCurrentMasterKey();
nextMasterKeyForNode = this.containerTokenSecretManager.getNextKey();
if (nextMasterKeyForNode != null) {
// nextMasterKeyForNode can be null if there is no outstanding key that
// is in the activation period.
if (nodeKnownMasterKey.getKeyId() != nextMasterKeyForNode.getKeyId()) {
shouldSendMasterKey = true;
}
}
if (shouldSendMasterKey) {
latestResponse.setMasterKey(nextMasterKeyForNode);
}
}
// 4. Send status to RMNode, saving the latest response.
this.rmContext.getDispatcher().getEventHandler().handle(
new RMNodeStatusEvent(nodeId, remoteNodeStatus.getNodeHealthStatus(),
remoteNodeStatus.getContainersStatuses(),
remoteNodeStatus.getKeepAliveApplications(), latestResponse));
remoteNodeStatus.getKeepAliveApplications(), latestResponse,
nextMasterKeyForNode));
nodeHeartBeatResponse.setHeartbeatResponse(latestResponse);
return nodeHeartBeatResponse;
@ -310,4 +333,7 @@ void refreshServiceAcls(Configuration configuration,
this.server.refreshServiceAcl(configuration, policyProvider);
}
protected boolean isSecurityEnabled() {
return UserGroupInformation.isSecurityEnabled();
}
}

View File

@ -28,6 +28,7 @@
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
/**
* Node managers information on available resources
@ -106,4 +107,6 @@ public interface RMNode {
public List<ApplicationId> getAppsToCleanup();
public HeartbeatResponse getLastHeartBeatResponse();
public MasterKey getCurrentMasterKey();
}

View File

@ -46,6 +46,7 @@
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.NodesListManagerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.NodesListManagerEventType;
@ -104,6 +105,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
private HeartbeatResponse latestHeartBeatResponse = recordFactory
.newRecordInstance(HeartbeatResponse.class);
private MasterKey currentMasterKey;
private static final StateMachineFactory<RMNodeImpl,
NodeState,
RMNodeEventType,
@ -155,7 +158,8 @@ RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
RMNodeEvent> stateMachine;
public RMNodeImpl(NodeId nodeId, RMContext context, String hostName,
int cmPort, int httpPort, Node node, Resource capability) {
int cmPort, int httpPort, Node node, Resource capability,
MasterKey masterKey) {
this.nodeId = nodeId;
this.context = context;
this.hostName = hostName;
@ -165,6 +169,7 @@ public RMNodeImpl(NodeId nodeId, RMContext context, String hostName,
this.nodeAddress = hostName + ":" + cmPort;
this.httpAddress = hostName + ":" + httpPort;
this.node = node;
this.currentMasterKey = masterKey;
this.nodeHealthStatus.setIsNodeHealthy(true);
this.nodeHealthStatus.setHealthReport("Healthy");
this.nodeHealthStatus.setLastHealthReportTime(System.currentTimeMillis());
@ -299,6 +304,17 @@ public HeartbeatResponse getLastHeartBeatResponse() {
}
}
@Override
public MasterKey getCurrentMasterKey() {
this.readLock.lock();
try {
return this.currentMasterKey;
} finally {
this.readLock.unlock();
}
}
public void handle(RMNodeEvent event) {
LOG.debug("Processing " + event.getNodeId() + " of type " + event.getType());
try {
@ -475,6 +491,7 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
// Switch the last heartbeatresponse.
rmNode.latestHeartBeatResponse = statusEvent.getLatestResponse();
rmNode.currentMasterKey = statusEvent.getCurrentMasterKey();
NodeHealthStatus remoteNodeHealthStatus =
statusEvent.getNodeHealthStatus();
@ -539,6 +556,7 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
// HeartBeat processing from our end is done, as node pulls the following
// lists before sending status-updates. Clear data-structures
// TODO: These lists could go to the NM multiple times, or never.
rmNode.containersToClean.clear();
rmNode.finishedApplications.clear();
@ -555,6 +573,7 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
// Switch the last heartbeatresponse.
rmNode.latestHeartBeatResponse = statusEvent.getLatestResponse();
rmNode.currentMasterKey = statusEvent.getCurrentMasterKey();
NodeHealthStatus remoteNodeHealthStatus = statusEvent.getNodeHealthStatus();
rmNode.setNodeHealthStatus(remoteNodeHealthStatus);
if (remoteNodeHealthStatus.getIsNodeHealthy()) {

View File

@ -25,6 +25,7 @@
import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
public class RMNodeStatusEvent extends RMNodeEvent {
@ -32,15 +33,17 @@ public class RMNodeStatusEvent extends RMNodeEvent {
private final List<ContainerStatus> containersCollection;
private final HeartbeatResponse latestResponse;
private final List<ApplicationId> keepAliveAppIds;
private final MasterKey currentMasterKey;
public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus,
List<ContainerStatus> collection, List<ApplicationId> keepAliveAppIds,
HeartbeatResponse latestResponse) {
HeartbeatResponse latestResponse, MasterKey currentMasterKey) {
super(nodeId, RMNodeEventType.STATUS_UPDATE);
this.nodeHealthStatus = nodeHealthStatus;
this.containersCollection = collection;
this.keepAliveAppIds = keepAliveAppIds;
this.latestResponse = latestResponse;
this.currentMasterKey = currentMasterKey;
}
public NodeHealthStatus getNodeHealthStatus() {
@ -58,4 +61,8 @@ public HeartbeatResponse getLatestResponse() {
public List<ApplicationId> getKeepAliveAppIds() {
return this.keepAliveAppIds;
}
public MasterKey getCurrentMasterKey() {
return this.currentMasterKey;
}
}

View File

@ -25,7 +25,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
/**
* This interface is the one implemented by the schedulers. It mainly extends
@ -38,9 +37,7 @@ public interface ResourceScheduler extends YarnScheduler, Recoverable {
/**
* Re-initialize the <code>ResourceScheduler</code>.
* @param conf configuration
* @param secretManager token-secret manager
* @throws IOException
*/
void reinitialize(Configuration conf,
ContainerTokenSecretManager secretManager, RMContext rmContext) throws IOException;
void reinitialize(Configuration conf, RMContext rmContext) throws IOException;
}

View File

@ -35,7 +35,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.Lock;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
@ -46,6 +45,7 @@
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
@ -75,7 +75,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@LimitedPrivate("yarn")
@Evolving
@ -126,7 +126,6 @@ public Configuration getConf() {
private CapacitySchedulerConfiguration conf;
private YarnConfiguration yarnConf;
private ContainerTokenSecretManager containerTokenSecretManager;
private RMContext rmContext;
private Map<String, CSQueue> queues = new ConcurrentHashMap<String, CSQueue>();
@ -163,8 +162,8 @@ public CapacitySchedulerConfiguration getConfiguration() {
}
@Override
public ContainerTokenSecretManager getContainerTokenSecretManager() {
return containerTokenSecretManager;
public RMContainerTokenSecretManager getContainerTokenSecretManager() {
return this.rmContext.getContainerTokenSecretManager();
}
@Override
@ -193,14 +192,12 @@ public Resource getClusterResources() {
}
@Override
public synchronized void reinitialize(Configuration conf,
ContainerTokenSecretManager containerTokenSecretManager, RMContext rmContext)
throws IOException {
public synchronized void
reinitialize(Configuration conf, RMContext rmContext) throws IOException {
if (!initialized) {
this.conf = new CapacitySchedulerConfiguration(conf);
this.minimumAllocation = this.conf.getMinimumAllocation();
this.maximumAllocation = this.conf.getMaximumAllocation();
this.containerTokenSecretManager = containerTokenSecretManager;
this.rmContext = rmContext;
initializeQueues(this.conf);
initialized = true;

View File

@ -21,7 +21,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
/**
* Read-only interface to {@link CapacityScheduler} context.
@ -33,7 +33,7 @@ public interface CapacitySchedulerContext {
Resource getMaximumResourceCapability();
ContainerTokenSecretManager getContainerTokenSecretManager();
RMContainerTokenSecretManager getContainerTokenSecretManager();
int getNumClusterNodes();

View File

@ -64,7 +64,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.BuilderUtils;
@Private
@ -104,7 +104,7 @@ public class LeafQueue implements CSQueue {
private final Resource maximumAllocation;
private final float minimumAllocationFactor;
private ContainerTokenSecretManager containerTokenSecretManager;
private RMContainerTokenSecretManager containerTokenSecretManager;
private Map<String, User> users = new HashMap<String, User>();

View File

@ -37,7 +37,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.BuilderUtils;
@Private
@ -51,7 +51,7 @@ public class AppSchedulable extends Schedulable {
private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
private static final Log LOG = LogFactory.getLog(AppSchedulable.class);
private FSQueue queue;
private ContainerTokenSecretManager containerTokenSecretManager;
private RMContainerTokenSecretManager containerTokenSecretManager;
public AppSchedulable(FairScheduler scheduler, FSSchedulerApp app, FSQueue queue) {
this.scheduler = scheduler;

View File

@ -73,7 +73,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@LimitedPrivate("yarn")
@Unstable
@ -82,7 +82,6 @@ public class FairScheduler implements ResourceScheduler {
private boolean initialized;
private FairSchedulerConfiguration conf;
private ContainerTokenSecretManager containerTokenSecretManager;
private RMContext rmContext;
private Resource minimumAllocation;
private Resource maximumAllocation;
@ -413,8 +412,8 @@ private void updateRunnability() {
}
}
public ContainerTokenSecretManager getContainerTokenSecretManager() {
return this.containerTokenSecretManager;
public RMContainerTokenSecretManager getContainerTokenSecretManager() {
return this.rmContext.getContainerTokenSecretManager();
}
public double getAppWeight(AppSchedulable app) {
@ -892,15 +891,11 @@ public void recover(RMState state) throws Exception {
}
@Override
public synchronized void reinitialize(Configuration conf,
ContainerTokenSecretManager containerTokenSecretManager,
RMContext rmContext)
throws IOException
{
public synchronized void
reinitialize(Configuration conf, RMContext rmContext) throws IOException {
if (!this.initialized) {
this.conf = new FairSchedulerConfiguration(conf);
this.rootMetrics = QueueMetrics.forQueue("root", null, true, conf);
this.containerTokenSecretManager = containerTokenSecretManager;
this.rmContext = rmContext;
this.clock = new SystemClock();
this.eventLog = new FairSchedulerEventLog();

View File

@ -83,7 +83,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.BuilderUtils;
@LimitedPrivate("yarn")
@ -97,7 +96,6 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
RecordFactoryProvider.getRecordFactory(null);
Configuration conf;
private ContainerTokenSecretManager containerTokenSecretManager;
private final static Container[] EMPTY_CONTAINER_ARRAY = new Container[] {};
private final static List<Container> EMPTY_CONTAINER_LIST = Arrays.asList(EMPTY_CONTAINER_ARRAY);
@ -193,14 +191,11 @@ public Resource getMaximumResourceCapability() {
}
@Override
public synchronized void reinitialize(Configuration conf,
ContainerTokenSecretManager containerTokenSecretManager,
RMContext rmContext)
throws IOException
public synchronized void
reinitialize(Configuration conf, RMContext rmContext) throws IOException
{
setConf(conf);
if (!this.initialized) {
this.containerTokenSecretManager = containerTokenSecretManager;
this.rmContext = rmContext;
this.minimumAllocation =
Resources.createResource(conf.getInt(
@ -543,8 +538,8 @@ private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application
// If security is enabled, send the container-tokens too.
if (UserGroupInformation.isSecurityEnabled()) {
containerToken =
containerTokenSecretManager.createContainerToken(containerId,
nodeId, capability);
this.rmContext.getContainerTokenSecretManager()
.createContainerToken(containerId, nodeId, capability);
if (containerToken == null) {
return i; // Try again later.
}

View File

@ -0,0 +1,153 @@
/**
* 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.security;
import java.util.Timer;
import java.util.TimerTask;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
/**
* SecretManager for ContainerTokens. This is RM-specific and rolls the
* master-keys every so often.
*
*/
public class RMContainerTokenSecretManager extends
BaseContainerTokenSecretManager {
private static Log LOG = LogFactory
.getLog(RMContainerTokenSecretManager.class);
private MasterKeyData nextMasterKey;
private final Timer timer;
private final long rollingInterval;
private final long activationDelay;
public RMContainerTokenSecretManager(Configuration conf) {
super(conf);
this.timer = new Timer();
this.rollingInterval = conf.getLong(
YarnConfiguration.RM_CONTAINER_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS,
YarnConfiguration.DEFAULT_RM_CONTAINER_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS) * 1000;
// Add an activation delay. This is to address the following race: RM may
// roll over master-key, scheduling may happen at some point of time, a
// container created with a password generated off new master key, but NM
// might not have come again to RM to update the shared secret: so AM has a
// valid password generated off new secret but NM doesn't know about the
// secret yet.
// Adding delay = 1.5 * expiry interval makes sure that all active NMs get
// the updated shared-key.
this.activationDelay =
(long) (conf.getLong(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS) * 1.5);
LOG.info("ContainerTokenKeyRollingInterval: " + this.rollingInterval
+ "ms and ContainerTokenKeyActivationDelay: " + this.activationDelay
+ "ms");
if (rollingInterval <= activationDelay * 2) {
throw new IllegalArgumentException(
YarnConfiguration.RM_CONTAINER_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS
+ " should be more than 2 X "
+ YarnConfiguration.RM_CONTAINER_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS);
}
}
public void start() {
rollMasterKey();
this.timer.scheduleAtFixedRate(new MasterKeyRoller(), rollingInterval,
rollingInterval);
}
public void stop() {
this.timer.cancel();
}
/**
* Creates a new master-key and sets it as the primary.
*/
@Private
protected void rollMasterKey() {
super.writeLock.lock();
try {
LOG.info("Rolling master-key for container-tokens");
if (this.currentMasterKey == null) { // Setting up for the first time.
this.currentMasterKey = createNewMasterKey();
} else {
this.nextMasterKey = createNewMasterKey();
this.timer.schedule(new NextKeyActivator(), this.activationDelay);
}
} finally {
super.writeLock.unlock();
}
}
@Private
public MasterKey getNextKey() {
super.readLock.lock();
try {
if (this.nextMasterKey == null) {
return null;
} else {
return this.nextMasterKey.getMasterKey();
}
} finally {
super.readLock.unlock();
}
}
/**
* Activate the new master-key
*/
@Private
protected void activateNextMasterKey() {
super.writeLock.lock();
try {
LOG.info("Activating next master key with id: "
+ this.nextMasterKey.getMasterKey().getKeyId());
this.currentMasterKey = this.nextMasterKey;
this.nextMasterKey = null;
} finally {
super.writeLock.unlock();
}
}
private class MasterKeyRoller extends TimerTask {
@Override
public void run() {
rollMasterKey();
}
}
private class NextKeyActivator extends TimerTask {
@Override
public void run() {
// Activation will happen after an absolute time interval. It will be good
// if we can force activation after an NM updates and acknowledges a
// roll-over. But that is only possible when we move to per-NM keys. TODO:
activateNextMasterKey();
}
}
}

View File

@ -30,6 +30,7 @@
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import com.google.common.collect.Lists;
@ -187,6 +188,11 @@ public List<ApplicationId> getAppsToCleanup() {
public HeartbeatResponse getLastHeartBeatResponse() {
return null;
}
@Override
public MasterKey getCurrentMasterKey() {
return null;
}
};
private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr) {

View File

@ -95,7 +95,7 @@ public static RMContext mockRMContext(int n, long time) {
rmDispatcher);
return new RMContextImpl(new MemStore(), rmDispatcher,
containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor,
null, null) {
null, null, null) {
@Override
public ConcurrentMap<ApplicationId, RMApp> getRMApps() {
return map;

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.yarn.server.resourcemanager;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import junit.framework.Assert;
@ -39,12 +38,10 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@ -201,7 +198,7 @@ public void testReconnectedNode() throws Exception {
conf.setQueues("default", new String[] {"default"});
conf.setCapacity("default", 100);
FifoScheduler fs = new FifoScheduler();
fs.reinitialize(conf, null, null);
fs.reinitialize(conf, null);
RMNode n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1);
RMNode n2 = MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 2);

View File

@ -17,29 +17,23 @@
*/
package org.apache.hadoop.yarn.server.resourcemanager;
import static junit.framework.Assert.assertEquals;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.ipc.TestRPC.TestImpl;
import org.apache.hadoop.ipc.TestRPC.TestProtocol;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.Keys;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.net.NetUtils;
import static org.mockito.Mockito.*;
import static junit.framework.Assert.*;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -48,7 +42,6 @@
* Tests {@link RMAuditLogger}.
*/
public class TestRMAuditLogger {
private static final Log LOG = LogFactory.getLog(TestRMAuditLogger.class);
private static final String USER = "test";
private static final String OPERATION = "oper";
private static final String TARGET = "tgt";

View File

@ -78,7 +78,7 @@ public void setUp() throws Exception {
rmContext =
new RMContextImpl(new MemStore(), rmDispatcher, null, null, null,
mock(DelegationTokenRenewer.class), null);
mock(DelegationTokenRenewer.class), null, null);
scheduler = mock(YarnScheduler.class);
doAnswer(
new Answer<Void>() {
@ -102,7 +102,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable {
new TestSchedulerEventDispatcher());
NodeId nodeId = BuilderUtils.newNodeId("localhost", 0);
node = new RMNodeImpl(nodeId, rmContext, null, 0, 0, null, null);
node = new RMNodeImpl(nodeId, rmContext, null, 0, 0, null, null, null);
}

View File

@ -62,7 +62,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSecretManager;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;

View File

@ -52,7 +52,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;

View File

@ -37,12 +37,12 @@
import org.apache.hadoop.yarn.server.resourcemanager.NodesListManager;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.NodeEventDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemStore;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.Records;
import org.junit.Before;
import org.junit.Test;
@ -71,7 +71,7 @@ public void setUp() {
// Dispatcher that processes events inline
Dispatcher dispatcher = new InlineDispatcher();
RMContext context = new RMContextImpl(new MemStore(), dispatcher, null,
null, null, null, null);
null, null, null, null, null);
dispatcher.register(SchedulerEventType.class,
new InlineDispatcher.EmptyEventHandler());
dispatcher.register(RMNodeEventType.class,
@ -82,8 +82,9 @@ public void setUp() {
nmLivelinessMonitor.start();
NodesListManager nodesListManager = new NodesListManager(context);
nodesListManager.init(conf);
ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager(conf);
RMContainerTokenSecretManager containerTokenSecretManager =
new RMContainerTokenSecretManager(conf);
containerTokenSecretManager.start();
resourceTrackerService = new ResourceTrackerService(context,
nodesListManager, nmLivelinessMonitor, containerTokenSecretManager);

View File

@ -42,13 +42,14 @@
import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemStore;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.Records;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@SuppressWarnings("rawtypes")
public class TestRMNMRPCResponseId {
private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
ResourceTrackerService resourceTrackerService;
@ -66,14 +67,14 @@ public void handle(Event event) {
});
RMContext context =
new RMContextImpl(new MemStore(), dispatcher, null, null, null,
null, null);
null, null, null);
dispatcher.register(RMNodeEventType.class,
new ResourceManager.NodeEventDispatcher(context));
NodesListManager nodesListManager = new NodesListManager(context);
Configuration conf = new Configuration();
nodesListManager.init(conf);
ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager(conf);
RMContainerTokenSecretManager containerTokenSecretManager =
new RMContainerTokenSecretManager(conf);
resourceTrackerService = new ResourceTrackerService(context,
nodesListManager, new NMLivelinessMonitor(dispatcher),
containerTokenSecretManager);

View File

@ -49,6 +49,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.junit.Before;
import org.junit.Test;
@ -119,9 +120,11 @@ public void setUp() throws Exception {
mock(ContainerAllocationExpirer.class);
AMLivelinessMonitor amLivelinessMonitor = mock(AMLivelinessMonitor.class);
AMLivelinessMonitor amFinishingMonitor = mock(AMLivelinessMonitor.class);
this.rmContext = new RMContextImpl(new MemStore(), rmDispatcher,
this.rmContext =
new RMContextImpl(new MemStore(), rmDispatcher,
containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor,
null, new ApplicationTokenSecretManager(conf));
null, new ApplicationTokenSecretManager(conf),
new RMContainerTokenSecretManager(conf));
rmDispatcher.register(RMAppAttemptEventType.class,
new TestApplicationAttemptEventDispatcher(this.rmContext));

View File

@ -72,6 +72,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.BuilderUtils;
import org.junit.After;
import org.junit.Before;
@ -154,9 +155,12 @@ public void setUp() throws Exception {
mock(ContainerAllocationExpirer.class);
amLivelinessMonitor = mock(AMLivelinessMonitor.class);
amFinishingMonitor = mock(AMLivelinessMonitor.class);
rmContext = new RMContextImpl(new MemStore(), rmDispatcher,
Configuration conf = new Configuration();
rmContext =
new RMContextImpl(new MemStore(), rmDispatcher,
containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor,
null, new ApplicationTokenSecretManager(new Configuration()));
null, new ApplicationTokenSecretManager(conf),
new RMContainerTokenSecretManager(conf));
scheduler = mock(YarnScheduler.class);
masterService = mock(ApplicationMasterService.class);
@ -174,7 +178,7 @@ public void setUp() throws Exception {
rmDispatcher.register(AMLauncherEventType.class,
new TestAMLauncherEventDispatcher());
rmDispatcher.init(new Configuration());
rmDispatcher.init(conf);
rmDispatcher.start();

View File

@ -35,6 +35,7 @@
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.Application;
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.Task;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
@ -44,6 +45,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -247,12 +249,13 @@ public void testRefreshQueues() throws Exception {
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
setupQueueConfiguration(conf);
cs.setConf(new YarnConfiguration());
cs.reinitialize(conf, null, null);
cs.reinitialize(conf, new RMContextImpl(null, null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf)));
checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
conf.setCapacity(A, 80f);
conf.setCapacity(B, 20f);
cs.reinitialize(conf, null,null);
cs.reinitialize(conf,null);
checkQueueCapacities(cs, 80f, 20f);
}
@ -343,7 +346,8 @@ public void testParseQueue() throws IOException {
conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".a.a1.b1", 100.0f);
conf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".a.a1.b1", 100.0f);
cs.reinitialize(conf, null, null);
cs.reinitialize(conf, new RMContextImpl(null, null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf)));
}
@Test
@ -353,7 +357,8 @@ public void testReconnectedNode() throws Exception {
setupQueueConfiguration(csConf);
CapacityScheduler cs = new CapacityScheduler();
cs.setConf(new YarnConfiguration());
cs.reinitialize(csConf, null, null);
cs.reinitialize(csConf, new RMContextImpl(null, null, null, null, null, null,
null, new RMContainerTokenSecretManager(csConf)));
RMNode n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1);
RMNode n2 = MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 2);

View File

@ -120,7 +120,7 @@ public void setUp() throws Exception {
CapacityScheduler.applicationComparator,
TestUtils.spyHook);
cs.reinitialize(csConf, null, rmContext);
cs.reinitialize(csConf, rmContext);
}
private static final String A = "a";

View File

@ -23,8 +23,8 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.junit.Test;
public class TestQueueParsing {
@ -42,7 +42,8 @@ public void testQueueParsing() throws Exception {
CapacityScheduler capacityScheduler = new CapacityScheduler();
capacityScheduler.setConf(conf);
capacityScheduler.reinitialize(conf, null, null);
capacityScheduler.reinitialize(conf, new RMContextImpl(null, null, null,
null, null, null, null, new RMContainerTokenSecretManager(conf)));
CSQueue a = capacityScheduler.getQueue("a");
Assert.assertEquals(0.10, a.getAbsoluteCapacity(), DELTA);
@ -138,7 +139,7 @@ public void testRootQueueParsing() throws Exception {
CapacityScheduler capacityScheduler = new CapacityScheduler();
capacityScheduler.setConf(new YarnConfiguration());
capacityScheduler.reinitialize(conf, null, null);
capacityScheduler.reinitialize(conf, null);
}
public void testMaxCapacity() throws Exception {
@ -161,7 +162,7 @@ public void testMaxCapacity() throws Exception {
try {
capacityScheduler = new CapacityScheduler();
capacityScheduler.setConf(new YarnConfiguration());
capacityScheduler.reinitialize(conf, null, null);
capacityScheduler.reinitialize(conf, null);
} catch (IllegalArgumentException iae) {
fail = true;
}
@ -173,7 +174,7 @@ public void testMaxCapacity() throws Exception {
// Now this should work
capacityScheduler = new CapacityScheduler();
capacityScheduler.setConf(new YarnConfiguration());
capacityScheduler.reinitialize(conf, null, null);
capacityScheduler.reinitialize(conf, null);
fail = false;
try {

View File

@ -47,6 +47,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
public class TestUtils {
private static final Log LOG = LogFactory.getLog(TestUtils.class);
@ -79,9 +80,11 @@ public EventHandler getEventHandler() {
ContainerAllocationExpirer cae =
new ContainerAllocationExpirer(nullDispatcher);
Configuration conf = new Configuration();
RMContext rmContext =
new RMContextImpl(null, nullDispatcher, cae, null, null, null,
new ApplicationTokenSecretManager(new Configuration()));
new ApplicationTokenSecretManager(conf),
new RMContainerTokenSecretManager(conf));
return rmContext;
}

View File

@ -100,7 +100,7 @@ public void setUp() throws IOException {
resourceManager = new ResourceManager(store);
resourceManager.init(conf);
((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
scheduler.reinitialize(conf, null, resourceManager.getRMContext());
scheduler.reinitialize(conf, resourceManager.getRMContext());
}
@After
@ -280,7 +280,7 @@ public void testSimpleContainerReservation() throws InterruptedException {
public void testUserAsDefaultQueue() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "true");
scheduler.reinitialize(conf, null, resourceManager.getRMContext());
scheduler.reinitialize(conf, resourceManager.getRMContext());
AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent(
createAppAttemptId(1, 1), "default", "user1");
scheduler.handle(appAddedEvent);
@ -288,7 +288,7 @@ public void testUserAsDefaultQueue() throws Exception {
assertEquals(0, scheduler.getQueueManager().getQueue("default").getApplications().size());
conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "false");
scheduler.reinitialize(conf, null, resourceManager.getRMContext());
scheduler.reinitialize(conf, resourceManager.getRMContext());
AppAddedSchedulerEvent appAddedEvent2 = new AppAddedSchedulerEvent(
createAppAttemptId(2, 1), "default", "user2");
scheduler.handle(appAddedEvent2);
@ -301,7 +301,7 @@ public void testUserAsDefaultQueue() throws Exception {
public void testFairShareWithMinAlloc() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
scheduler.reinitialize(conf, null, resourceManager.getRMContext());
scheduler.reinitialize(conf, resourceManager.getRMContext());
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>");
@ -407,7 +407,7 @@ public void testAppAdditionAndRemoval() throws Exception {
public void testAllocationFileParsing() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
scheduler.reinitialize(conf, null, resourceManager.getRMContext());
scheduler.reinitialize(conf, resourceManager.getRMContext());
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>");
@ -510,7 +510,7 @@ public void testAllocationFileParsing() throws Exception {
public void testBackwardsCompatibleAllocationFileParsing() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
scheduler.reinitialize(conf, null, resourceManager.getRMContext());
scheduler.reinitialize(conf, resourceManager.getRMContext());
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>");
@ -613,7 +613,7 @@ public void testBackwardsCompatibleAllocationFileParsing() throws Exception {
public void testIsStarvedForMinShare() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
scheduler.reinitialize(conf, null, resourceManager.getRMContext());
scheduler.reinitialize(conf, resourceManager.getRMContext());
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>");
@ -672,7 +672,7 @@ else if (p.getName().equals("queueB")) {
public void testIsStarvedForFairShare() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
scheduler.reinitialize(conf, null, resourceManager.getRMContext());
scheduler.reinitialize(conf, resourceManager.getRMContext());
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>");
@ -736,7 +736,7 @@ else if (p.getName().equals("queueB")) {
public void testChoiceOfPreemptedContainers() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE + ".allocation.file", ALLOC_FILE);
scheduler.reinitialize(conf, null, resourceManager.getRMContext());
scheduler.reinitialize(conf, resourceManager.getRMContext());
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>");
@ -871,7 +871,7 @@ public void testPreemptionDecision() throws Exception {
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
MockClock clock = new MockClock();
scheduler.setClock(clock);
scheduler.reinitialize(conf, null, resourceManager.getRMContext());
scheduler.reinitialize(conf, resourceManager.getRMContext());
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>");

View File

@ -87,10 +87,10 @@ public void testFifoSchedulerCapacityWhenNoNMs() {
public void testAppAttemptMetrics() throws Exception {
AsyncDispatcher dispatcher = new InlineDispatcher();
RMContext rmContext = new RMContextImpl(null, dispatcher, null,
null, null, null, null);
null, null, null, null, null);
FifoScheduler schedular = new FifoScheduler();
schedular.reinitialize(new Configuration(), null, rmContext);
schedular.reinitialize(new Configuration(), rmContext);
ApplicationId appId = BuilderUtils.newApplicationId(200, 1);
ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(

View File

@ -46,6 +46,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.util.StringHelper;
import org.apache.hadoop.yarn.webapp.WebApps;
@ -159,7 +160,7 @@ public static RMContext mockRMContext(int numApps, int racks, int numNodes,
deactivatedNodesMap.put(node.getHostName(), node);
}
return new RMContextImpl(new MemStore(), null, null, null, null,
null, null) {
null, null, null) {
@Override
public ConcurrentMap<ApplicationId, RMApp> getRMApps() {
return applicationsMaps;
@ -199,7 +200,8 @@ public static CapacityScheduler mockCapacityScheduler() throws IOException {
CapacityScheduler cs = new CapacityScheduler();
cs.setConf(new YarnConfiguration());
cs.reinitialize(conf, null, null);
cs.reinitialize(conf, new RMContextImpl(null, null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf)));
return cs;
}
@ -275,7 +277,7 @@ public static FifoScheduler mockFifoScheduler() throws Exception {
FifoScheduler fs = new FifoScheduler();
fs.setConf(new YarnConfiguration());
fs.reinitialize(conf, null, null);
fs.reinitialize(conf, null);
return fs;
}

View File

@ -145,7 +145,7 @@ public void testNodesDefaultWithUnHealthyNode() throws JSONException,
nodeHealth.setHealthReport("test health report");
nodeHealth.setIsNodeHealthy(false);
node.handle(new RMNodeStatusEvent(nm3.getNodeId(), nodeHealth,
new ArrayList<ContainerStatus>(), null, null));
new ArrayList<ContainerStatus>(), null, null, null));
rm.NMwaitForState(nm3.getNodeId(), NodeState.UNHEALTHY);
ClientResponse response =
@ -360,7 +360,7 @@ public void testNodesQueryHealthyAndState() throws JSONException, Exception {
nodeHealth.setHealthReport("test health report");
nodeHealth.setIsNodeHealthy(false);
node.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeHealth,
new ArrayList<ContainerStatus>(), null, null));
new ArrayList<ContainerStatus>(), null, null, null));
rm.NMwaitForState(nm1.getNodeId(), NodeState.UNHEALTHY);
ClientResponse response = r.path("ws").path("v1").path("cluster")

View File

@ -21,7 +21,6 @@
import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
import java.net.ServerSocket;
import java.net.UnknownHostException;
import org.apache.commons.logging.Log;
@ -51,7 +50,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.service.CompositeService;
@ -290,10 +288,9 @@ protected void doSecureLogin() throws IOException {
@Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
ContainerTokenSecretManager containerTokenSecretManager) {
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
return new NodeStatusUpdaterImpl(context, dispatcher,
healthChecker, metrics, containerTokenSecretManager) {
healthChecker, metrics) {
@Override
protected ResourceTracker getRMClient() {
final ResourceTrackerService rt = resourceManager

View File

@ -46,6 +46,7 @@
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.SecurityUtil;
@ -86,6 +87,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.BuilderUtils;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Records;
@ -220,7 +222,7 @@ public void testMaliceUser() throws IOException, InterruptedException {
Resource modifiedResource = BuilderUtils.newResource(2048);
ContainerTokenIdentifier modifiedIdentifier = new ContainerTokenIdentifier(
dummyIdentifier.getContainerID(), dummyIdentifier.getNmHostAddress(),
modifiedResource, Long.MAX_VALUE);
modifiedResource, Long.MAX_VALUE, 0);
Token<ContainerTokenIdentifier> modifiedToken = new Token<ContainerTokenIdentifier>(
modifiedIdentifier.getBytes(), containerToken.getPassword().array(),
new Text(containerToken.getKind()), new Text(containerToken
@ -250,12 +252,17 @@ public Void run() {
Assert.assertEquals(
java.lang.reflect.UndeclaredThrowableException.class
.getCanonicalName(), e.getClass().getCanonicalName());
Assert.assertEquals(RemoteException.class.getCanonicalName(), e
.getCause().getClass().getCanonicalName());
Assert.assertEquals(
"org.apache.hadoop.security.token.SecretManager$InvalidToken",
((RemoteException) e.getCause()).getClassName());
Assert.assertTrue(e
.getCause()
.getMessage()
.contains(
"DIGEST-MD5: digest response format violation. "
+ "Mismatched response."));
.matches(
"Given Container container_\\d*_\\d*_\\d\\d_\\d*"
+ " seems to have an illegally generated token."));
}
return null;
}
@ -331,12 +338,15 @@ public ContainerManager run() {
unauthorizedUser = UserGroupInformation
.createRemoteUser(containerID.toString());
RMContainerTokenSecretManager containerTokenSecreteManager =
resourceManager.getRMContainerTokenSecretManager();
final ContainerTokenIdentifier newTokenId =
new ContainerTokenIdentifier(tokenId.getContainerID(),
tokenId.getNmHostAddress(), tokenId.getResource(),
System.currentTimeMillis() - 1);
System.currentTimeMillis() - 1,
containerTokenSecreteManager.getCurrentKey().getKeyId());
byte[] passowrd =
resourceManager.getContainerTokenSecretManager().createPassword(
containerTokenSecreteManager.createPassword(
newTokenId);
// Create a valid token by using the key from the RM.
token = new Token<ContainerTokenIdentifier>(