|
@@ -30,7 +30,6 @@ 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.api.records.ContainerLaunchContext;
|
|
|
-import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResource;
|
|
@@ -43,7 +42,6 @@ import org.apache.hadoop.yarn.security.NMTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
@@ -83,12 +81,11 @@ public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
LOG = LogFactory.getLog(TestQueuingContainerManager.class);
|
|
|
}
|
|
|
|
|
|
- HasResources hasResources = null;
|
|
|
boolean shouldDeleteWait = false;
|
|
|
|
|
|
@Override
|
|
|
- protected ContainerManagerImpl
|
|
|
- createContainerManager(DeletionService delSrvc) {
|
|
|
+ protected ContainerManagerImpl createContainerManager(
|
|
|
+ DeletionService delSrvc) {
|
|
|
return new QueuingContainerManagerImpl(context, exec, delSrvc,
|
|
|
nodeStatusUpdater, metrics, dirsHandler) {
|
|
|
|
|
@@ -123,21 +120,35 @@ public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
|
|
|
- Container container, boolean stopRequest, NMTokenIdentifier identifier) throws YarnException {
|
|
|
- if(container == null || container.getUser().equals("Fail")){
|
|
|
+ protected void authorizeGetAndStopContainerRequest(
|
|
|
+ ContainerId containerId, Container container, boolean stopRequest,
|
|
|
+ NMTokenIdentifier identifier) throws YarnException {
|
|
|
+ if (container == null || container.getUser().equals("Fail")) {
|
|
|
throw new YarnException("Reject this container");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- protected ContainersMonitor createContainersMonitor(ContainerExecutor
|
|
|
- exec) {
|
|
|
+ protected ContainersMonitor createContainersMonitor(
|
|
|
+ ContainerExecutor exec) {
|
|
|
return new ContainersMonitorImpl(exec, dispatcher, this.context) {
|
|
|
+ // Define resources available for containers to be executed.
|
|
|
@Override
|
|
|
- public boolean hasResourcesAvailable(
|
|
|
- ContainersMonitorImpl.ProcessTreeInfo pti) {
|
|
|
- return hasResources.decide(this.context, pti.getContainerId());
|
|
|
+ public long getPmemAllocatedForContainers() {
|
|
|
+ return 2048 * 1024 * 1024L;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public long getVmemAllocatedForContainers() {
|
|
|
+ float pmemRatio = getConfig().getFloat(
|
|
|
+ YarnConfiguration.NM_VMEM_PMEM_RATIO,
|
|
|
+ YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
|
|
|
+ return (long) (pmemRatio * getPmemAllocatedForContainers());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public long getVCoresAllocatedForContainers() {
|
|
|
+ return 2;
|
|
|
}
|
|
|
};
|
|
|
}
|
|
@@ -172,12 +183,6 @@ public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
public void setup() throws IOException {
|
|
|
super.setup();
|
|
|
shouldDeleteWait = false;
|
|
|
- hasResources = new HasResources() {
|
|
|
- @Override
|
|
|
- public boolean decide(Context context, ContainerId cId) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- };
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -256,31 +261,16 @@ public class TestQueuingContainerManager extends TestContainerManager {
|
|
|
containerLaunchContext,
|
|
|
createContainerToken(createContainerId(3), DUMMY_RM_IDENTIFIER,
|
|
|
context.getNodeId(),
|
|
|
- user, context.getContainerTokenSecretManager())));
|
|
|
+ user, BuilderUtils.newResource(1024, 1),
|
|
|
+ context.getContainerTokenSecretManager(), null,
|
|
|
+ ExecutionType.GUARANTEED)));
|
|
|
StartContainersRequest allRequests =
|
|
|
StartContainersRequest.newInstance(list);
|
|
|
|
|
|
- // Plugin to simulate that the Node is full
|
|
|
- // It only allows 1 container to run at a time.
|
|
|
- hasResources = new HasResources() {
|
|
|
- @Override
|
|
|
- public boolean decide(Context context, ContainerId cId) {
|
|
|
- int nOpp = ((QueuingContainerManagerImpl) containerManager)
|
|
|
- .getNumAllocatedOpportunisticContainers();
|
|
|
- int nGuar = ((QueuingContainerManagerImpl) containerManager)
|
|
|
- .getNumAllocatedGuaranteedContainers();
|
|
|
- boolean val = (nOpp + nGuar < 1);
|
|
|
- System.out.println("\nHasResources : [" + cId + "]," +
|
|
|
- "Opp[" + nOpp + "], Guar[" + nGuar + "], [" + val + "]\n");
|
|
|
- return val;
|
|
|
- }
|
|
|
- };
|
|
|
-
|
|
|
containerManager.startContainers(allRequests);
|
|
|
|
|
|
- BaseContainerManagerTest.waitForContainerState(containerManager,
|
|
|
- createContainerId(3),
|
|
|
- ContainerState.COMPLETE, 40);
|
|
|
+ Thread.sleep(10000);
|
|
|
+
|
|
|
List<ContainerId> statList = new ArrayList<ContainerId>();
|
|
|
for (int i = 0; i < 4; i++) {
|
|
|
statList.add(createContainerId(i));
|