|
@@ -27,7 +27,9 @@ import java.util.Map;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Collections;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.List;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
@@ -40,6 +42,7 @@ import javax.ws.rs.core.Response.Status;
|
|
|
|
|
|
import org.apache.commons.lang3.EnumUtils;
|
|
|
import org.apache.commons.lang3.StringUtils;
|
|
|
+import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
@@ -47,6 +50,11 @@ import org.apache.hadoop.security.authorize.AuthorizationException;
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
|
|
|
import org.apache.hadoop.util.Sets;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ReservationId;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
@@ -65,10 +73,11 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
|
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationTimeout;
|
|
|
+import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
|
|
|
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
|
|
|
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
|
|
|
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
|
|
|
import org.apache.hadoop.yarn.api.records.QueueACL;
|
|
|
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
|
|
|
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
|
|
|
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
@@ -78,7 +87,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystemTestUtil;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
@@ -117,10 +125,19 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.StatisticsItemInfo;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewReservation;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationListInfo;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDefinitionInfo;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationRequestInfo;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationRequestsInfo;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateResponseInfo;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteResponseInfo;
|
|
|
+import org.apache.hadoop.yarn.server.router.RouterServerUtil;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.PartitionInfo;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.RMQueueAclInfo;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServices;
|
|
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
|
|
import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
|
|
|
import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
|
|
@@ -134,6 +151,11 @@ import org.mockito.Mockito;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+
|
|
|
+import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEFAULT;
|
|
|
+import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEFAULT_FULL;
|
|
|
+import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEDICATED;
|
|
|
+import static org.apache.hadoop.yarn.server.router.webapp.BaseRouterWebServicesTest.QUEUE_DEDICATED_FULL;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
@@ -153,19 +175,16 @@ public class MockDefaultRequestInterceptorREST
|
|
|
private Map<ApplicationId, ApplicationReport> applicationMap = new HashMap<>();
|
|
|
public static final String APP_STATE_RUNNING = "RUNNING";
|
|
|
|
|
|
- private static final String QUEUE_DEFAULT = "default";
|
|
|
- private static final String QUEUE_DEFAULT_FULL = CapacitySchedulerConfiguration.ROOT +
|
|
|
- CapacitySchedulerConfiguration.DOT + QUEUE_DEFAULT;
|
|
|
- private static final String QUEUE_DEDICATED = "dedicated";
|
|
|
- public static final String QUEUE_DEDICATED_FULL = CapacitySchedulerConfiguration.ROOT +
|
|
|
- CapacitySchedulerConfiguration.DOT + QUEUE_DEDICATED;
|
|
|
-
|
|
|
// duration(milliseconds), 1mins
|
|
|
public static final long DURATION = 60*1000;
|
|
|
|
|
|
// Containers 4
|
|
|
public static final int NUM_CONTAINERS = 4;
|
|
|
|
|
|
+ private Map<ReservationId, SubClusterId> reservationMap = new HashMap<>();
|
|
|
+ private AtomicLong resCounter = new AtomicLong();
|
|
|
+ private MockRM mockRM = null;
|
|
|
+
|
|
|
private void validateRunning() throws ConnectException {
|
|
|
if (!isRunning) {
|
|
|
throw new ConnectException("RM is stopped");
|
|
@@ -859,44 +878,191 @@ public class MockDefaultRequestInterceptorREST
|
|
|
" Please try again with a valid reservable queue.");
|
|
|
}
|
|
|
|
|
|
- MockRM mockRM = setupResourceManager();
|
|
|
+ ReservationId reservationID =
|
|
|
+ ReservationId.parseReservationId(reservationId);
|
|
|
|
|
|
- ReservationId reservationID = ReservationId.parseReservationId(reservationId);
|
|
|
- ReservationSystem reservationSystem = mockRM.getReservationSystem();
|
|
|
- reservationSystem.synchronizePlan(QUEUE_DEDICATED_FULL, true);
|
|
|
+ if (!reservationMap.containsKey(reservationID)) {
|
|
|
+ throw new NotFoundException("reservationId with id: " + reservationId + " not found");
|
|
|
+ }
|
|
|
|
|
|
- // Generate reserved resources
|
|
|
ClientRMService clientService = mockRM.getClientRMService();
|
|
|
|
|
|
- // arrival time from which the resource(s) can be allocated.
|
|
|
- long arrival = Time.now();
|
|
|
-
|
|
|
- // deadline by when the resource(s) must be allocated.
|
|
|
- // The reason for choosing 1.05 is because this gives an integer
|
|
|
- // DURATION * 0.05 = 3000(ms)
|
|
|
- // deadline = arrival + 3000ms
|
|
|
- long deadline = (long) (arrival + 1.05 * DURATION);
|
|
|
-
|
|
|
- // In this test of reserved resources, we will apply for 4 containers (1 core, 1GB memory)
|
|
|
- // arrival = Time.now(), and make sure deadline - arrival > duration,
|
|
|
- // the current setting is greater than 3000ms
|
|
|
- ReservationSubmissionRequest submissionRequest =
|
|
|
- ReservationSystemTestUtil.createSimpleReservationRequest(
|
|
|
- reservationID, NUM_CONTAINERS, arrival, deadline, DURATION);
|
|
|
- clientService.submitReservation(submissionRequest);
|
|
|
-
|
|
|
// listReservations
|
|
|
ReservationListRequest request = ReservationListRequest.newInstance(
|
|
|
- queue, reservationID.toString(), startTime, endTime, includeResourceAllocations);
|
|
|
+ queue, reservationId, startTime, endTime, includeResourceAllocations);
|
|
|
ReservationListResponse resRespInfo = clientService.listReservations(request);
|
|
|
ReservationListInfo resResponse =
|
|
|
new ReservationListInfo(resRespInfo, includeResourceAllocations);
|
|
|
|
|
|
- if (mockRM != null) {
|
|
|
- mockRM.stop();
|
|
|
+ return Response.status(Status.OK).entity(resResponse).build();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Response createNewReservation(HttpServletRequest hsr)
|
|
|
+ throws AuthorizationException, IOException, InterruptedException {
|
|
|
+
|
|
|
+ if (!isRunning) {
|
|
|
+ throw new RuntimeException("RM is stopped");
|
|
|
+ }
|
|
|
+
|
|
|
+ ReservationId resId = ReservationId.newInstance(Time.now(), resCounter.incrementAndGet());
|
|
|
+ LOG.info("Allocated new reservationId: {}.", resId);
|
|
|
+
|
|
|
+ NewReservation reservationId = new NewReservation(resId.toString());
|
|
|
+ return Response.status(Status.OK).entity(reservationId).build();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Response submitReservation(ReservationSubmissionRequestInfo resContext,
|
|
|
+ HttpServletRequest hsr) throws AuthorizationException, IOException, InterruptedException {
|
|
|
+
|
|
|
+ if (!isRunning) {
|
|
|
+ throw new RuntimeException("RM is stopped");
|
|
|
}
|
|
|
|
|
|
- return Response.status(Status.OK).entity(resResponse).build();
|
|
|
+ ReservationId reservationId = ReservationId.parseReservationId(resContext.getReservationId());
|
|
|
+ ReservationDefinitionInfo definitionInfo = resContext.getReservationDefinition();
|
|
|
+ ReservationDefinition definition =
|
|
|
+ RouterServerUtil.convertReservationDefinition(definitionInfo);
|
|
|
+ ReservationSubmissionRequest request = ReservationSubmissionRequest.newInstance(
|
|
|
+ definition, resContext.getQueue(), reservationId);
|
|
|
+ submitReservation(request);
|
|
|
+
|
|
|
+ LOG.info("Reservation submitted: {}.", reservationId);
|
|
|
+
|
|
|
+ SubClusterId subClusterId = getSubClusterId();
|
|
|
+ reservationMap.put(reservationId, subClusterId);
|
|
|
+
|
|
|
+ return Response.status(Status.ACCEPTED).build();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void submitReservation(ReservationSubmissionRequest request) {
|
|
|
+ try {
|
|
|
+ // synchronize plan
|
|
|
+ ReservationSystem reservationSystem = mockRM.getReservationSystem();
|
|
|
+ reservationSystem.synchronizePlan(QUEUE_DEDICATED_FULL, true);
|
|
|
+ // Generate reserved resources
|
|
|
+ ClientRMService clientService = mockRM.getClientRMService();
|
|
|
+ clientService.submitReservation(request);
|
|
|
+ } catch (IOException | YarnException e) {
|
|
|
+ throw new RuntimeException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Response updateReservation(ReservationUpdateRequestInfo resContext,
|
|
|
+ HttpServletRequest hsr) throws AuthorizationException, IOException, InterruptedException {
|
|
|
+
|
|
|
+ if (resContext == null || resContext.getReservationId() == null ||
|
|
|
+ resContext.getReservationDefinition() == null) {
|
|
|
+ return Response.status(Status.BAD_REQUEST).build();
|
|
|
+ }
|
|
|
+
|
|
|
+ String resId = resContext.getReservationId();
|
|
|
+ ReservationId reservationId = ReservationId.parseReservationId(resId);
|
|
|
+
|
|
|
+ if (!reservationMap.containsKey(reservationId)) {
|
|
|
+ throw new NotFoundException("reservationId with id: " + reservationId + " not found");
|
|
|
+ }
|
|
|
+
|
|
|
+ // Generate reserved resources
|
|
|
+ updateReservation(resContext);
|
|
|
+
|
|
|
+ ReservationUpdateResponseInfo resRespInfo = new ReservationUpdateResponseInfo();
|
|
|
+ return Response.status(Status.OK).entity(resRespInfo).build();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void updateReservation(ReservationUpdateRequestInfo resContext) throws IOException {
|
|
|
+
|
|
|
+ if (resContext == null) {
|
|
|
+ throw new BadRequestException("Input ReservationSubmissionContext should not be null");
|
|
|
+ }
|
|
|
+
|
|
|
+ ReservationDefinitionInfo resInfo = resContext.getReservationDefinition();
|
|
|
+ if (resInfo == null) {
|
|
|
+ throw new BadRequestException("Input ReservationDefinition should not be null");
|
|
|
+ }
|
|
|
+
|
|
|
+ ReservationRequestsInfo resReqsInfo = resInfo.getReservationRequests();
|
|
|
+ if (resReqsInfo == null || resReqsInfo.getReservationRequest() == null
|
|
|
+ || resReqsInfo.getReservationRequest().isEmpty()) {
|
|
|
+ throw new BadRequestException("The ReservationDefinition should " +
|
|
|
+ "contain at least one ReservationRequest");
|
|
|
+ }
|
|
|
+
|
|
|
+ if (resContext.getReservationId() == null) {
|
|
|
+ throw new BadRequestException("Update operations must specify an existing ReservaitonId");
|
|
|
+ }
|
|
|
+
|
|
|
+ ReservationRequestInterpreter[] values = ReservationRequestInterpreter.values();
|
|
|
+ ReservationRequestInterpreter requestInterpreter =
|
|
|
+ values[resReqsInfo.getReservationRequestsInterpreter()];
|
|
|
+ List<ReservationRequest> list = new ArrayList<>();
|
|
|
+
|
|
|
+ for (ReservationRequestInfo resReqInfo : resReqsInfo.getReservationRequest()) {
|
|
|
+ ResourceInfo rInfo = resReqInfo.getCapability();
|
|
|
+ Resource capability = Resource.newInstance(rInfo.getMemorySize(), rInfo.getvCores());
|
|
|
+ int numContainers = resReqInfo.getNumContainers();
|
|
|
+ int minConcurrency = resReqInfo.getMinConcurrency();
|
|
|
+ long duration = resReqInfo.getDuration();
|
|
|
+ ReservationRequest rr = ReservationRequest.newInstance(
|
|
|
+ capability, numContainers, minConcurrency, duration);
|
|
|
+ list.add(rr);
|
|
|
+ }
|
|
|
+
|
|
|
+ ReservationRequests reqs = ReservationRequests.newInstance(list, requestInterpreter);
|
|
|
+ ReservationDefinition rDef = ReservationDefinition.newInstance(
|
|
|
+ resInfo.getArrival(), resInfo.getDeadline(), reqs,
|
|
|
+ resInfo.getReservationName(), resInfo.getRecurrenceExpression(),
|
|
|
+ Priority.newInstance(resInfo.getPriority()));
|
|
|
+ ReservationUpdateRequest request = ReservationUpdateRequest.newInstance(
|
|
|
+ rDef, ReservationId.parseReservationId(resContext.getReservationId()));
|
|
|
+
|
|
|
+ ClientRMService clientService = mockRM.getClientRMService();
|
|
|
+ try {
|
|
|
+ clientService.updateReservation(request);
|
|
|
+ } catch (YarnException ex) {
|
|
|
+ throw new RuntimeException(ex);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Response deleteReservation(ReservationDeleteRequestInfo resContext, HttpServletRequest hsr)
|
|
|
+ throws AuthorizationException, IOException, InterruptedException {
|
|
|
+ if (!isRunning) {
|
|
|
+ throw new RuntimeException("RM is stopped");
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ String resId = resContext.getReservationId();
|
|
|
+ ReservationId reservationId = ReservationId.parseReservationId(resId);
|
|
|
+
|
|
|
+ if (!reservationMap.containsKey(reservationId)) {
|
|
|
+ throw new NotFoundException("reservationId with id: " + reservationId + " not found");
|
|
|
+ }
|
|
|
+
|
|
|
+ ReservationDeleteRequest reservationDeleteRequest =
|
|
|
+ ReservationDeleteRequest.newInstance(reservationId);
|
|
|
+ ClientRMService clientService = mockRM.getClientRMService();
|
|
|
+ clientService.deleteReservation(reservationDeleteRequest);
|
|
|
+
|
|
|
+ ReservationDeleteResponseInfo resRespInfo = new ReservationDeleteResponseInfo();
|
|
|
+ reservationMap.remove(reservationId);
|
|
|
+
|
|
|
+ return Response.status(Status.OK).entity(resRespInfo).build();
|
|
|
+ } catch (YarnException e) {
|
|
|
+ throw new RuntimeException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public MockRM getMockRM() {
|
|
|
+ return mockRM;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public void setMockRM(MockRM mockResourceManager) {
|
|
|
+ this.mockRM = mockResourceManager;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -939,7 +1105,7 @@ public class MockDefaultRequestInterceptorREST
|
|
|
public RMQueueAclInfo checkUserAccessToQueue(String queue, String username,
|
|
|
String queueAclType, HttpServletRequest hsr) throws AuthorizationException {
|
|
|
|
|
|
- ResourceManager mockRM = mock(ResourceManager.class);
|
|
|
+ ResourceManager mockResourceManager = mock(ResourceManager.class);
|
|
|
Configuration conf = new YarnConfiguration();
|
|
|
|
|
|
ResourceScheduler mockScheduler = new CapacityScheduler() {
|
|
@@ -959,8 +1125,9 @@ public class MockDefaultRequestInterceptorREST
|
|
|
}
|
|
|
};
|
|
|
|
|
|
- when(mockRM.getResourceScheduler()).thenReturn(mockScheduler);
|
|
|
- MockRMWebServices webSvc = new MockRMWebServices(mockRM, conf, mock(HttpServletResponse.class));
|
|
|
+ when(mockResourceManager.getResourceScheduler()).thenReturn(mockScheduler);
|
|
|
+ MockRMWebServices webSvc = new MockRMWebServices(mockResourceManager, conf,
|
|
|
+ mock(HttpServletResponse.class));
|
|
|
return webSvc.checkUserAccessToQueue(queue, username, queueAclType, hsr);
|
|
|
}
|
|
|
|