|
@@ -105,6 +105,8 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
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.ExecutionTypeRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.NMToken;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeReport;
|
|
@@ -868,6 +870,196 @@ public class TestRMContainerAllocator {
|
|
|
allocator.close();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test A MapReduce job can be configured to opt out of oversubscription,
|
|
|
+ * that is, it always wait for guaranteed resources to execute its tasks.
|
|
|
+ * This is done by setting up a MapReduce job with 2 mappers and 1 reducers
|
|
|
+ * and capturing all ResourceRequests sent from the AM to RM, then checking
|
|
|
+ * if all ResourceRequests are guaranteed and their enforceExecutionType is
|
|
|
+ * true.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testMapReduceOptingOutOversubscription() throws Exception {
|
|
|
+ List<ResourceRequest> resourceRequests = captureResourceRequests(true);
|
|
|
+
|
|
|
+ for(ResourceRequest resourceRequest : resourceRequests) {
|
|
|
+ ExecutionTypeRequest executionTypeRequest =
|
|
|
+ resourceRequest.getExecutionTypeRequest();
|
|
|
+ if (!executionTypeRequest.equals(ExecutionTypeRequest.newInstance(
|
|
|
+ ExecutionType.GUARANTEED, true))) {
|
|
|
+ Assert.fail("The execution type of ResourceRequest " + resourceRequest +
|
|
|
+ " is not guaranteed or not enforced.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test a MapReduce job can be configured to opt in oversubscription (
|
|
|
+ * true by default). This is done by setting up a MapReduce job with 2
|
|
|
+ * mappers and 1 reducers and capturing all ResourceRequests sent from
|
|
|
+ * the AM to RM, then checking if all ResourceRequests are guaranteed
|
|
|
+ * but their enforceExecutionType is always set to false.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testMapReduceOptingInOversubscription() throws Exception {
|
|
|
+ List<ResourceRequest> resourceRequests = captureResourceRequests(false);
|
|
|
+
|
|
|
+ for(ResourceRequest resourceRequest : resourceRequests) {
|
|
|
+ ExecutionTypeRequest executionTypeRequest =
|
|
|
+ resourceRequest.getExecutionTypeRequest();
|
|
|
+ if (!executionTypeRequest.equals(ExecutionTypeRequest.newInstance(
|
|
|
+ ExecutionType.GUARANTEED, false))) {
|
|
|
+ Assert.fail("The execution type of ResourceRequest " + resourceRequest +
|
|
|
+ " is not guaranteed or it is enforced.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Set up a mapreduce job with 2 mappers and 1 reducer and return
|
|
|
+ * all ResourceRequests sent from the AM to RM.
|
|
|
+ */
|
|
|
+ private List<ResourceRequest> captureResourceRequests(
|
|
|
+ boolean optOutOfOversubscription) throws Exception {
|
|
|
+ List<ResourceRequest> resourceRequests = new ArrayList<>();
|
|
|
+
|
|
|
+ final Configuration conf = new Configuration();
|
|
|
+ conf.setBoolean(MRJobConfig.MR_OVERSUBSCRIPTION_OPT_OUT,
|
|
|
+ optOutOfOversubscription);
|
|
|
+
|
|
|
+ // start the resource manager
|
|
|
+ final MyResourceManager rm = new MyResourceManager(conf);
|
|
|
+ rm.start();
|
|
|
+ DrainDispatcher rmDispatcher = (DrainDispatcher) rm.getRMContext()
|
|
|
+ .getDispatcher();
|
|
|
+
|
|
|
+ // submit an application
|
|
|
+ RMApp rmApp = rm.submitApp(1024);
|
|
|
+ rm.drainEvents();
|
|
|
+
|
|
|
+ MockNM amNodeManager = rm.registerNode("amNM:1234", 11264);
|
|
|
+ amNodeManager.nodeHeartbeat(true);
|
|
|
+ rm.drainEvents();
|
|
|
+
|
|
|
+ final ApplicationAttemptId appAttemptId = rmApp.getCurrentAppAttempt()
|
|
|
+ .getAppAttemptId();
|
|
|
+ rm.sendAMLaunched(appAttemptId);
|
|
|
+ rm.drainEvents();
|
|
|
+
|
|
|
+ // start the MR AM and wait until it is in running state
|
|
|
+ MRApp mrApp = new MRApp(appAttemptId, ContainerId.newContainerId(
|
|
|
+ appAttemptId, 0), 2, 1, false,
|
|
|
+ this.getClass().getName(), true, 1) {
|
|
|
+ @Override
|
|
|
+ protected Dispatcher createDispatcher() {
|
|
|
+ return new DrainDispatcher();
|
|
|
+ }
|
|
|
+ protected ContainerAllocator createContainerAllocator(
|
|
|
+ ClientService clientService, AppContext context) {
|
|
|
+ return new MyContainerAllocator(rm, appAttemptId, context);
|
|
|
+ };
|
|
|
+ };
|
|
|
+ mrApp.submit(conf);
|
|
|
+
|
|
|
+ Job job = mrApp.getContext().getAllJobs().entrySet().iterator().next()
|
|
|
+ .getValue();
|
|
|
+ DrainDispatcher amDispatcher = (DrainDispatcher) mrApp.getDispatcher();
|
|
|
+ MyContainerAllocator allocator = (MyContainerAllocator) mrApp
|
|
|
+ .getContainerAllocator();
|
|
|
+ mrApp.waitForInternalState((JobImpl)job, JobStateInternal.RUNNING);
|
|
|
+ amDispatcher.await();
|
|
|
+
|
|
|
+ // wait until all attempts request for containers
|
|
|
+ for (Task t : job.getTasks().values()) {
|
|
|
+ mrApp.waitForInternalState((TaskAttemptImpl) t.getAttempts().values()
|
|
|
+ .iterator().next(), TaskAttemptStateInternal.UNASSIGNED);
|
|
|
+ }
|
|
|
+ amDispatcher.await();
|
|
|
+
|
|
|
+ // send map resource requests to RM
|
|
|
+ allocator.schedule();
|
|
|
+ rm.drainEvents();
|
|
|
+ for (ResourceRequest rr : rm.getMyFifoScheduler().lastAsk) {
|
|
|
+ resourceRequests.add(ResourceRequest.newInstance(
|
|
|
+ rr.getPriority(), rr.getResourceName(), rr.getCapability(),
|
|
|
+ rr.getNumContainers(), rr.getRelaxLocality(),
|
|
|
+ rr.getNodeLabelExpression(), rr.getExecutionTypeRequest()));
|
|
|
+ }
|
|
|
+
|
|
|
+ // wait for both map tasks to be running
|
|
|
+ amNodeManager.nodeHeartbeat(true);
|
|
|
+ rm.drainEvents();
|
|
|
+
|
|
|
+ allocator.schedule();
|
|
|
+ rm.drainEvents();
|
|
|
+ for (ResourceRequest rr : rm.getMyFifoScheduler().lastAsk) {
|
|
|
+ resourceRequests.add(ResourceRequest.newInstance(
|
|
|
+ rr.getPriority(), rr.getResourceName(), rr.getCapability(),
|
|
|
+ rr.getNumContainers(), rr.getRelaxLocality(),
|
|
|
+ rr.getNodeLabelExpression(), rr.getExecutionTypeRequest()));
|
|
|
+ }
|
|
|
+
|
|
|
+ for (Task t : job.getTasks().values()) {
|
|
|
+ if (t.getType() == TaskType.MAP) {
|
|
|
+ mrApp.waitForState(t, TaskState.RUNNING);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // finish both map tasks so that the reduce task can be scheduled
|
|
|
+ Iterator<Task> it = job.getTasks().values().iterator();
|
|
|
+ finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 2);
|
|
|
+ allocator.schedule();
|
|
|
+ rm.drainEvents();
|
|
|
+ for (ResourceRequest rr : rm.getMyFifoScheduler().lastAsk) {
|
|
|
+ resourceRequests.add(ResourceRequest.newInstance(
|
|
|
+ rr.getPriority(), rr.getResourceName(), rr.getCapability(),
|
|
|
+ rr.getNumContainers(), rr.getRelaxLocality(),
|
|
|
+ rr.getNodeLabelExpression(), rr.getExecutionTypeRequest()));
|
|
|
+ }
|
|
|
+
|
|
|
+ // send the reduce resource requests to RM
|
|
|
+ allocator.schedule();
|
|
|
+ rm.drainEvents();
|
|
|
+ for (ResourceRequest rr : rm.getMyFifoScheduler().lastAsk) {
|
|
|
+ resourceRequests.add(ResourceRequest.newInstance(
|
|
|
+ rr.getPriority(), rr.getResourceName(), rr.getCapability(),
|
|
|
+ rr.getNumContainers(), rr.getRelaxLocality(),
|
|
|
+ rr.getNodeLabelExpression(), rr.getExecutionTypeRequest()));
|
|
|
+ }
|
|
|
+
|
|
|
+ // wait for the reduce task to be running
|
|
|
+ amNodeManager.nodeHeartbeat(true);
|
|
|
+ rm.drainEvents();
|
|
|
+
|
|
|
+ allocator.schedule();
|
|
|
+ rm.drainEvents();
|
|
|
+ for (ResourceRequest rr : rm.getMyFifoScheduler().lastAsk) {
|
|
|
+ resourceRequests.add(ResourceRequest.newInstance(
|
|
|
+ rr.getPriority(), rr.getResourceName(), rr.getCapability(),
|
|
|
+ rr.getNumContainers(), rr.getRelaxLocality(),
|
|
|
+ rr.getNodeLabelExpression(), rr.getExecutionTypeRequest()));
|
|
|
+ }
|
|
|
+
|
|
|
+ for (Task t : job.getTasks().values()) {
|
|
|
+ if (t.getType() == TaskType.REDUCE) {
|
|
|
+ mrApp.waitForState(t, TaskState.RUNNING);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // finish the reduce task
|
|
|
+ finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 1);
|
|
|
+ allocator.schedule();
|
|
|
+ rm.drainEvents();
|
|
|
+ for (ResourceRequest rr : rm.getMyFifoScheduler().lastAsk) {
|
|
|
+ resourceRequests.add(ResourceRequest.newInstance(
|
|
|
+ rr.getPriority(), rr.getResourceName(), rr.getCapability(),
|
|
|
+ rr.getNumContainers(), rr.getRelaxLocality(),
|
|
|
+ rr.getNodeLabelExpression(), rr.getExecutionTypeRequest()));
|
|
|
+ }
|
|
|
+
|
|
|
+ return resourceRequests;
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testMapReduceScheduling() throws Exception {
|
|
|
|