|
@@ -26,6 +26,9 @@ import java.util.HashMap;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -97,6 +100,9 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
private final RMContainerTokenSecretManager containerTokenSecretManager;
|
|
|
private final NMTokenSecretManagerInRM nmTokenSecretManager;
|
|
|
|
|
|
+ private final ReadLock readLock;
|
|
|
+ private final WriteLock writeLock;
|
|
|
+
|
|
|
private long nextHeartBeatInterval;
|
|
|
private Server server;
|
|
|
private InetSocketAddress resourceTrackerAddress;
|
|
@@ -107,7 +113,7 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
|
|
|
private boolean isDistributedNodeLabelsConf;
|
|
|
private boolean isDelegatedCentralizedNodeLabelsConf;
|
|
|
- private volatile DynamicResourceConfiguration drConf;
|
|
|
+ private DynamicResourceConfiguration drConf;
|
|
|
|
|
|
public ResourceTrackerService(RMContext rmContext,
|
|
|
NodesListManager nodesListManager,
|
|
@@ -120,7 +126,9 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
this.nmLivelinessMonitor = nmLivelinessMonitor;
|
|
|
this.containerTokenSecretManager = containerTokenSecretManager;
|
|
|
this.nmTokenSecretManager = nmTokenSecretManager;
|
|
|
-
|
|
|
+ ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
|
|
+ this.readLock = lock.readLock();
|
|
|
+ this.writeLock = lock.writeLock();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -160,7 +168,6 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
}
|
|
|
|
|
|
loadDynamicResourceConfiguration(conf);
|
|
|
-
|
|
|
super.serviceInit(conf);
|
|
|
}
|
|
|
|
|
@@ -176,6 +183,9 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
InputStream drInputStream = this.rmContext.getConfigurationProvider()
|
|
|
.getConfigurationInputStream(conf,
|
|
|
YarnConfiguration.DR_CONFIGURATION_FILE);
|
|
|
+ // write lock here on drConfig is unnecessary as here get called at
|
|
|
+ // ResourceTrackerService get initiated and other read and write
|
|
|
+ // operations haven't started yet.
|
|
|
if (drInputStream != null) {
|
|
|
this.drConf = new DynamicResourceConfiguration(conf, drInputStream);
|
|
|
} else {
|
|
@@ -192,7 +202,12 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
*/
|
|
|
public void updateDynamicResourceConfiguration(
|
|
|
DynamicResourceConfiguration conf) {
|
|
|
- this.drConf = conf;
|
|
|
+ this.writeLock.lock();
|
|
|
+ try {
|
|
|
+ this.drConf = conf;
|
|
|
+ } finally {
|
|
|
+ this.writeLock.unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -233,6 +248,7 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
if (this.server != null) {
|
|
|
this.server.stop();
|
|
|
}
|
|
|
+
|
|
|
super.serviceStop();
|
|
|
}
|
|
|
|
|
@@ -331,16 +347,18 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
}
|
|
|
|
|
|
// check if node's capacity is load from dynamic-resources.xml
|
|
|
- String[] nodes = this.drConf.getNodes();
|
|
|
String nid = nodeId.toString();
|
|
|
|
|
|
- if (nodes != null && Arrays.asList(nodes).contains(nid)) {
|
|
|
- capability.setMemory(this.drConf.getMemoryPerNode(nid));
|
|
|
- capability.setVirtualCores(this.drConf.getVcoresPerNode(nid));
|
|
|
+ Resource dynamicLoadCapability = loadNodeResourceFromDRConfiguration(nid);
|
|
|
+ if (dynamicLoadCapability != null) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Resource for node: " + nid + " is adjusted to " +
|
|
|
- capability + " due to settings in dynamic-resources.xml.");
|
|
|
+ LOG.debug("Resource for node: " + nid + " is adjusted from: " +
|
|
|
+ capability + " to: " + dynamicLoadCapability +
|
|
|
+ " due to settings in dynamic-resources.xml.");
|
|
|
}
|
|
|
+ capability = dynamicLoadCapability;
|
|
|
+ // sync back with new resource.
|
|
|
+ response.setResource(capability);
|
|
|
}
|
|
|
|
|
|
// Check if this node has minimum allocations
|
|
@@ -536,8 +554,17 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // 6. Send Container Queuing Limits back to the Node. This will be used by
|
|
|
- // the node to truncate the number of Containers queued for execution.
|
|
|
+ // 6. check if node's capacity is load from dynamic-resources.xml
|
|
|
+ // if so, send updated resource back to NM.
|
|
|
+ String nid = nodeId.toString();
|
|
|
+ Resource capability = loadNodeResourceFromDRConfiguration(nid);
|
|
|
+ // sync back with new resource if not null.
|
|
|
+ if (capability != null) {
|
|
|
+ nodeHeartBeatResponse.setResource(capability);
|
|
|
+ }
|
|
|
+
|
|
|
+ // 7. Send Container Queuing Limits back to the Node. This will be used by
|
|
|
+ // the node to truncate the number of Containers queued for execution.
|
|
|
if (this.rmContext.getNodeManagerQueueLimitCalculator() != null) {
|
|
|
nodeHeartBeatResponse.setContainerQueuingLimit(
|
|
|
this.rmContext.getNodeManagerQueueLimitCalculator()
|
|
@@ -629,6 +656,22 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private Resource loadNodeResourceFromDRConfiguration(String nodeId) {
|
|
|
+ // check if node's capacity is loaded from dynamic-resources.xml
|
|
|
+ this.readLock.lock();
|
|
|
+ try {
|
|
|
+ String[] nodes = this.drConf.getNodes();
|
|
|
+ if (nodes != null && Arrays.asList(nodes).contains(nodeId)) {
|
|
|
+ return Resource.newInstance(this.drConf.getMemoryPerNode(nodeId),
|
|
|
+ this.drConf.getVcoresPerNode(nodeId));
|
|
|
+ } else {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ this.readLock.unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* resolving the network topology.
|
|
|
* @param hostName the hostname of this node.
|