|
@@ -21,6 +21,7 @@ import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.nio.ByteBuffer;
|
|
|
+import java.util.Arrays;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
@@ -62,6 +63,7 @@ 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.resourcemanager.nodelabels.NodeLabelsUtils;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration;
|
|
|
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.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
|
|
@@ -105,6 +107,7 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
|
|
|
private boolean isDistributedNodeLabelsConf;
|
|
|
private boolean isDelegatedCentralizedNodeLabelsConf;
|
|
|
+ private volatile DynamicResourceConfiguration drConf;
|
|
|
|
|
|
public ResourceTrackerService(RMContext rmContext,
|
|
|
NodesListManager nodesListManager,
|
|
@@ -139,11 +142,11 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
}
|
|
|
|
|
|
minAllocMb = conf.getInt(
|
|
|
- YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
|
|
|
- YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
|
|
|
+ YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
|
|
|
minAllocVcores = conf.getInt(
|
|
|
- YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
|
|
|
- YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
|
|
|
+ YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
|
|
|
|
|
|
minimumNodeManagerVersion = conf.get(
|
|
|
YarnConfiguration.RM_NODEMANAGER_MINIMUM_VERSION,
|
|
@@ -156,9 +159,42 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
YarnConfiguration.isDelegatedCentralizedNodeLabelConfiguration(conf);
|
|
|
}
|
|
|
|
|
|
+ loadDynamicResourceConfiguration(conf);
|
|
|
+
|
|
|
super.serviceInit(conf);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Load DynamicResourceConfiguration from dynamic-resources.xml.
|
|
|
+ * @param conf
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ public void loadDynamicResourceConfiguration(Configuration conf)
|
|
|
+ throws IOException {
|
|
|
+ try {
|
|
|
+ // load dynamic-resources.xml
|
|
|
+ InputStream drInputStream = this.rmContext.getConfigurationProvider()
|
|
|
+ .getConfigurationInputStream(conf,
|
|
|
+ YarnConfiguration.DR_CONFIGURATION_FILE);
|
|
|
+ if (drInputStream != null) {
|
|
|
+ this.drConf = new DynamicResourceConfiguration(conf, drInputStream);
|
|
|
+ } else {
|
|
|
+ this.drConf = new DynamicResourceConfiguration(conf);
|
|
|
+ }
|
|
|
+ } catch (Exception e) {
|
|
|
+ throw new IOException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Update DynamicResourceConfiguration with new configuration.
|
|
|
+ * @param conf
|
|
|
+ */
|
|
|
+ public void updateDynamicResourceConfiguration(
|
|
|
+ DynamicResourceConfiguration conf) {
|
|
|
+ this.drConf = conf;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
protected void serviceStart() throws Exception {
|
|
|
super.serviceStart();
|
|
@@ -166,15 +202,14 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
// security is enabled, so no secretManager.
|
|
|
Configuration conf = getConfig();
|
|
|
YarnRPC rpc = YarnRPC.create(conf);
|
|
|
- this.server =
|
|
|
- rpc.getServer(ResourceTracker.class, this, resourceTrackerAddress,
|
|
|
- conf, null,
|
|
|
- conf.getInt(YarnConfiguration.RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT,
|
|
|
- YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT));
|
|
|
-
|
|
|
+ this.server = rpc.getServer(
|
|
|
+ ResourceTracker.class, this, resourceTrackerAddress, conf, null,
|
|
|
+ conf.getInt(YarnConfiguration.RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT,
|
|
|
+ YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT));
|
|
|
+
|
|
|
// Enable service authorization?
|
|
|
if (conf.getBoolean(
|
|
|
- CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
|
|
|
+ CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
|
|
|
false)) {
|
|
|
InputStream inputStream =
|
|
|
this.rmContext.getConfigurationProvider()
|
|
@@ -185,12 +220,12 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
}
|
|
|
refreshServiceAcls(conf, RMPolicyProvider.getInstance());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
this.server.start();
|
|
|
conf.updateConnectAddr(YarnConfiguration.RM_BIND_HOST,
|
|
|
- YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
|
|
|
- YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_ADDRESS,
|
|
|
- server.getListenerAddress());
|
|
|
+ YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
|
|
|
+ YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_ADDRESS,
|
|
|
+ server.getListenerAddress());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -295,6 +330,19 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
return response;
|
|
|
}
|
|
|
|
|
|
+ // 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));
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Resource for node: " + nid + " is adjusted to " +
|
|
|
+ capability + " due to settings in dynamic-resources.xml.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
// Check if this node has minimum allocations
|
|
|
if (capability.getMemory() < minAllocMb
|
|
|
|| capability.getVirtualCores() < minAllocVcores) {
|
|
@@ -311,7 +359,7 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
response.setContainerTokenMasterKey(containerTokenSecretManager
|
|
|
.getCurrentKey());
|
|
|
response.setNMTokenMasterKey(nmTokenSecretManager
|
|
|
- .getCurrentKey());
|
|
|
+ .getCurrentKey());
|
|
|
|
|
|
RMNode rmNode = new RMNodeImpl(nodeId, rmContext, host, cmPort, httpPort,
|
|
|
resolve(host), capability, nodeManagerVersion);
|