You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2016/02/10 19:25:08 UTC

[1/2] hadoop git commit: YARN-4420. Add REST API for List Reservations (Sean Po via curino)

Repository: hadoop
Updated Branches:
  refs/heads/trunk bc425a623 -> c3641ed3b


YARN-4420. Add REST API for List Reservations (Sean Po via curino)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b706cbc1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b706cbc1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b706cbc1

Branch: refs/heads/trunk
Commit: b706cbc1bc0ab3572c01676fe7365df21eda7ffa
Parents: 4ef1324
Author: = <cu...@apache.org>
Authored: Wed Feb 10 10:19:26 2016 -0800
Committer: = <cu...@apache.org>
Committed: Wed Feb 10 10:19:26 2016 -0800

----------------------------------------------------------------------
 .../resourcemanager/webapp/RMWebServices.java   |  56 ++
 .../webapp/dao/ReservationDefinitionInfo.java   |  10 +
 .../webapp/dao/ReservationIdInfo.java           |  64 +++
 .../webapp/dao/ReservationInfo.java             | 105 ++++
 .../webapp/dao/ReservationListInfo.java         |  53 ++
 .../webapp/dao/ReservationRequestInfo.java      |   9 +
 .../webapp/dao/ReservationRequestsInfo.java     |  11 +
 .../webapp/dao/ResourceAllocationInfo.java      |  72 +++
 .../webapp/TestRMWebServicesReservation.java    | 542 ++++++++++++++++++-
 .../src/test/resources/submit-reservation.json  |   6 +-
 10 files changed, 903 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b706cbc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 2ccd673..0ed8a4e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -39,6 +39,7 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
 import javax.ws.rs.FormParam;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
@@ -82,6 +83,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
@@ -156,6 +159,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDefinitionInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteResponseInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationListInfo;
 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.ReservationSubmissionResponseInfo;
@@ -2168,4 +2172,56 @@ public class RMWebServices extends WebServices {
     return request;
   }
 
+  /**
+   * Function to retrieve a list of all the reservations.
+   */
+  @GET
+  @Path("/reservation/list")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public Response listReservation(
+          @QueryParam("queue") @DefaultValue("default") String queue,
+          @QueryParam("reservation-id") @DefaultValue("") String reservationId,
+          @QueryParam("start-time") @DefaultValue("0") long startTime,
+          @QueryParam("end-time") @DefaultValue("-1") long endTime,
+          @QueryParam("include-resource-allocations") @DefaultValue("false")
+          boolean includeResourceAllocations, @Context HttpServletRequest hsr)
+          throws Exception {
+    init();
+
+    final ReservationListRequest request = ReservationListRequest.newInstance(
+          queue, reservationId, startTime, endTime, includeResourceAllocations);
+
+    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
+    if (callerUGI == null) {
+      throw new AuthorizationException("Unable to obtain user name, "
+              + "user not authenticated");
+    }
+    if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) {
+      String msg = "The default static user cannot carry out this operation.";
+      return Response.status(Status.FORBIDDEN).entity(msg).build();
+    }
+
+    ReservationListResponse resRespInfo;
+    try {
+      resRespInfo = callerUGI.doAs(
+          new PrivilegedExceptionAction<ReservationListResponse>() {
+            @Override
+            public ReservationListResponse run() throws IOException,
+                    YarnException {
+              return rm.getClientRMService().listReservations(request);
+            }
+          });
+    } catch (UndeclaredThrowableException ue) {
+      if (ue.getCause() instanceof YarnException) {
+        throw new BadRequestException(ue.getCause().getMessage());
+      }
+      LOG.info("List reservation request failed", ue);
+      throw ue;
+    }
+
+    ReservationListInfo resResponse = new ReservationListInfo(resRespInfo,
+            includeResourceAllocations);
+    return Response.status(Status.OK).entity(resResponse).build();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b706cbc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDefinitionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDefinitionInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDefinitionInfo.java
index ff82e75..71ee924 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDefinitionInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDefinitionInfo.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
 
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
@@ -46,6 +48,14 @@ public class ReservationDefinitionInfo {
 
   }
 
+  public ReservationDefinitionInfo(ReservationDefinition definition) {
+    arrival = definition.getArrival();
+    deadline = definition.getDeadline();
+    reservationName = definition.getReservationName();
+    reservationRequests = new ReservationRequestsInfo(definition
+            .getReservationRequests());
+  }
+
   public long getArrival() {
     return arrival;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b706cbc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationIdInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationIdInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationIdInfo.java
new file mode 100644
index 0000000..3a2596a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationIdInfo.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import org.apache.hadoop.yarn.api.records.ReservationId;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple class that represent a reservation ID.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationIdInfo {
+  @XmlElement(name = "cluster-timestamp")
+  private long clusterTimestamp;
+
+  @XmlElement(name = "reservation-id")
+  private long reservationId;
+
+  public ReservationIdInfo() {
+    this.clusterTimestamp = 0;
+    this.reservationId = 0;
+  }
+
+  public ReservationIdInfo(ReservationId reservationId) {
+    this.clusterTimestamp = reservationId.getClusterTimestamp();
+    this.reservationId = reservationId.getId();
+  }
+
+  public long getClusterTimestamp() {
+    return this.clusterTimestamp;
+  }
+
+  public void setClusterTimestamp(long newClusterTimestamp) {
+    this.clusterTimestamp = newClusterTimestamp;
+  }
+
+  public long getReservationId() {
+    return this.reservationId;
+  }
+
+  public void setReservationId(long newReservationId) {
+    this.reservationId = newReservationId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b706cbc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationInfo.java
new file mode 100644
index 0000000..1a31a8b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationInfo.java
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import org.apache.hadoop.yarn.api.records.ReservationAllocationState;
+import org.apache.hadoop.yarn.api.records.ResourceAllocationRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Simple class that represent a reservation.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationInfo {
+
+  @XmlElement(name = "acceptance-time")
+  private long acceptanceTime;
+
+  private String user;
+
+  @XmlElement(name = "resource-allocations")
+  private List<ResourceAllocationInfo> resourceAllocations = new ArrayList<>();
+
+  @XmlElement(name = "reservation-id")
+  private ReservationIdInfo reservationId;
+
+  @XmlElement(name = "reservation-definition")
+  private ReservationDefinitionInfo reservationDefinition;
+
+
+  public ReservationInfo() {
+    acceptanceTime = 0;
+    user = "";
+
+    reservationDefinition = new ReservationDefinitionInfo();
+    reservationId = new ReservationIdInfo();
+  }
+
+  public ReservationInfo(ReservationAllocationState allocation, boolean
+        includeResourceAllocations) throws
+        Exception {
+    acceptanceTime = allocation.getAcceptanceTime();
+    user = allocation.getUser();
+
+    if (includeResourceAllocations) {
+      List<ResourceAllocationRequest> requests = allocation
+              .getResourceAllocationRequests();
+      for (ResourceAllocationRequest request : requests) {
+        resourceAllocations.add(new ResourceAllocationInfo(new
+                ReservationInterval(request.getStartTime(), request
+                .getEndTime()), request.getCapability()));
+      }
+    }
+
+    reservationId = new ReservationIdInfo(allocation.getReservationId());
+    reservationDefinition = new ReservationDefinitionInfo(
+            allocation.getReservationDefinition());
+  }
+
+  public String getUser() {
+    return user;
+  }
+
+  public void setUser(String newUser) {
+    user = newUser;
+  }
+
+  public long getAcceptanceTime() {
+    return acceptanceTime;
+  }
+
+  public List<ResourceAllocationInfo> getResourceAllocations() {
+    return resourceAllocations;
+  }
+
+  public ReservationIdInfo getReservationId() {
+    return reservationId;
+  }
+
+  public ReservationDefinitionInfo getReservationDefinition() {
+    return reservationDefinition;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b706cbc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationListInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationListInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationListInfo.java
new file mode 100644
index 0000000..25df67a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationListInfo.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
+import org.apache.hadoop.yarn.api.records.ReservationAllocationState;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Simple class that represent a list of reservations.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReservationListInfo {
+  @XmlElement(name = "reservations")
+  private List<ReservationInfo> reservations;
+
+  public ReservationListInfo() {
+    reservations = new ArrayList<>();
+  }
+
+  public ReservationListInfo(ReservationListResponse response,
+        boolean includeResourceAllocations) throws Exception {
+    this();
+
+    for (ReservationAllocationState allocation :
+            response.getReservationAllocationState()) {
+      reservations.add(new ReservationInfo(allocation,
+              includeResourceAllocations));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b706cbc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestInfo.java
index ab82f3f..e1d6fa3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestInfo.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
 
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
@@ -43,6 +45,13 @@ public class ReservationRequestInfo {
 
   }
 
+  public ReservationRequestInfo(ReservationRequest request) {
+    capability = new ResourceInfo(request.getCapability());
+    minConcurrency = request.getConcurrency();
+    duration = request.getDuration();
+    numContainers = request.getNumContainers();
+  }
+
   public ResourceInfo getCapability() {
     return capability;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b706cbc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestsInfo.java
index 42cdb0e..f37da1b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationRequestsInfo.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
 
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+
 import java.util.ArrayList;
 
 import javax.xml.bind.annotation.XmlAccessType;
@@ -42,6 +45,14 @@ public class ReservationRequestsInfo {
 
   }
 
+  public ReservationRequestsInfo(ReservationRequests requests) {
+    reservationRequest = new ArrayList<>();
+    for (ReservationRequest request : requests.getReservationResources()) {
+      reservationRequest.add(new ReservationRequestInfo(request));
+    }
+    reservationRequestsInterpreter = requests.getInterpreter().ordinal();
+  }
+
   public int getReservationRequestsInterpreter() {
     return reservationRequestsInterpreter;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b706cbc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceAllocationInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceAllocationInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceAllocationInfo.java
new file mode 100644
index 0000000..9ed9f51
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceAllocationInfo.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationInterval;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Simple class that represent a resource allocation.
+ */
+@XmlRootElement(name = "resource-allocation")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ResourceAllocationInfo {
+  private ResourceInfo resource;
+  private long startTime;
+  private long endTime;
+
+  public ResourceAllocationInfo() {
+    resource = new ResourceInfo();
+    startTime = -1;
+    endTime = -1;
+  }
+
+  public ResourceAllocationInfo(ReservationInterval interval, Resource res) {
+    this.resource = new ResourceInfo(res);
+    this.startTime = interval.getStartTime();
+    this.endTime = interval.getEndTime();
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public void setStartTime(long newStartTime) {
+    this.startTime = newStartTime;
+  }
+
+  public long getEndTime() {
+    return endTime;
+  }
+
+  public void setEndTime(long newEndTime) {
+    this.endTime = newEndTime;
+  }
+
+  public ResourceInfo getResource() {
+    return resource;
+  }
+
+  public void setResource(ResourceInfo newResource) {
+    this.resource = newResource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b706cbc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java
index bb0db81..72705d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.FileWriter;
@@ -54,8 +55,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedule
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
 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.util.Clock;
+import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
+import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.After;
@@ -82,17 +86,22 @@ import com.sun.jersey.test.framework.WebAppDescriptor;
 
 @RunWith(Parameterized.class)
 public class TestRMWebServicesReservation extends JerseyTestBase {
-  private static MockRM rm;
 
-  private static Injector injector;
   private String webserviceUserName = "testuser";
-
   private boolean setAuthFilter = false;
 
+  private static MockRM rm;
+  private static Injector injector;
+
+  private static final int MINIMUM_RESOURCE_DURATION = 1000000;
+  private static final Clock clock = new UTCClock();
   private static final String TEST_DIR = new File(System.getProperty(
       "test.build.data", "/tmp")).getAbsolutePath();
   private static final String FS_ALLOC_FILE = new File(TEST_DIR,
       "test-fs-queues.xml").getAbsolutePath();
+  // This is what is used in the test resource files.
+  private static final String DEFAULT_QUEUE = "dedicated";
+  private static final String LIST_RESERVATION_PATH = "reservation/list";
 
   public static class GuiceServletConfig extends GuiceServletContextListener {
 
@@ -318,13 +327,9 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
   }
 
   @Test
-  public void testSubmitReservation() throws JSONException, Exception {
+  public void testSubmitReservation() throws Exception {
     rm.start();
-    for (int i = 0; i < 100; i++) {
-      MockNM amNodeManager =
-          rm.registerNode("127.0.0." + i + ":1234", 100 * 1024);
-      amNodeManager.nodeHeartbeat(true);
-    }
+    setupCluster(100);
     ReservationId rid =
         testSubmissionReservationHelper("reservation/submit",
             MediaType.APPLICATION_JSON);
@@ -335,14 +340,10 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
   }
 
   @Test
-  public void testFailedSubmitReservation() throws JSONException, Exception {
+  public void testFailedSubmitReservation() throws Exception {
     rm.start();
     // setup a cluster too small to accept the reservation
-    for (int i = 0; i < 1; i++) {
-      MockNM amNodeManager =
-          rm.registerNode("127.0.0." + i + ":1234", 100 * 1024);
-      amNodeManager.nodeHeartbeat(true);
-    }
+    setupCluster(1);
     ReservationId rid =
         testSubmissionReservationHelper("reservation/submit",
             MediaType.APPLICATION_JSON);
@@ -353,11 +354,7 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
   @Test
   public void testUpdateReservation() throws JSONException, Exception {
     rm.start();
-    for (int i = 0; i < 100; i++) {
-      MockNM amNodeManager =
-          rm.registerNode("127.0.0." + i + ":1234", 100 * 1024);
-      amNodeManager.nodeHeartbeat(true);
-    }
+    setupCluster(100);
     ReservationId rid =
         testSubmissionReservationHelper("reservation/submit",
             MediaType.APPLICATION_JSON);
@@ -371,6 +368,452 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
   }
 
   @Test
+  public void testTimeIntervalRequestListReservation() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    long time = clock.getTime() + MINIMUM_RESOURCE_DURATION;
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time, "res_1", 1);
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION,
+            "res_2", 2);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("start-time", Long.toString((long) (time * 0.9)))
+            .queryParam("end-time", Long.toString(time + (long) (0.9 *
+                    MINIMUM_RESOURCE_DURATION)))
+            .queryParam("include-resource-allocations", "true")
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    JSONObject reservations = json.getJSONObject("reservations");
+
+    testRDLHelper(reservations);
+
+    String reservationName = reservations.getJSONObject
+            ("reservation-definition").getString("reservation-name");
+    assertEquals(reservationName, "res_1");
+
+    rm.stop();
+  }
+
+  @Test
+  public void testSameTimeIntervalRequestListReservation() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    long time = clock.getTime() + MINIMUM_RESOURCE_DURATION;
+
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time, "res_1", 1);
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION,
+            "res_2", 2);
+
+    String timeParam = Long.toString(time + MINIMUM_RESOURCE_DURATION / 2);
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("start-time", timeParam)
+            .queryParam("end-time", timeParam)
+            .queryParam("include-resource-allocations", "true")
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    JSONObject reservations = json.getJSONObject("reservations");
+
+    testRDLHelper(reservations);
+
+    String reservationName = reservations.getJSONObject
+            ("reservation-definition").getString("reservation-name");
+    assertEquals(reservationName, "res_1");
+
+    rm.stop();
+  }
+  @Test
+  public void testInvalidTimeIntervalRequestListReservation() throws
+          Exception {
+    rm.start();
+    setupCluster(100);
+
+    long time = clock.getTime() + MINIMUM_RESOURCE_DURATION;
+
+    ReservationId res1 = testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time, "res_1", 1);
+    ReservationId res2 = testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION,
+            "res_2", 2);
+
+    WebResource resource;
+    resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("start-time", "-100")
+            .queryParam("end-time", "-100")
+            .queryParam("include-resource-allocations", "true")
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    JSONArray reservations = json.getJSONArray("reservations");
+
+    assertEquals(2, reservations.length());
+
+    testRDLHelper(reservations.getJSONObject(0));
+    testRDLHelper(reservations.getJSONObject(1));
+
+    rm.stop();
+  }
+
+  @Test
+  public void testInvalidEndTimeRequestListReservation() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    long time = clock.getTime() + MINIMUM_RESOURCE_DURATION;
+
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time, "res_1", 1);
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION,
+            "res_2", 2);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("start-time", Long.toString((long) (time +
+                    MINIMUM_RESOURCE_DURATION * 1.3)))
+            .queryParam("end-time", "-1")
+            .queryParam("include-resource-allocations", "true")
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    JSONObject reservations = json.getJSONObject("reservations");
+
+    testRDLHelper(reservations);
+
+    String reservationName = reservations.getJSONObject
+            ("reservation-definition").getString("reservation-name");
+    assertEquals(reservationName, "res_2");
+
+    rm.stop();
+  }
+
+  @Test
+  public void testEmptyEndTimeRequestListReservation() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    long time = clock.getTime() + MINIMUM_RESOURCE_DURATION;
+
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time, "res_1", 1);
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION,
+            "res_2", 2);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("start-time", new Long((long) (time +
+                    MINIMUM_RESOURCE_DURATION * 1.3)).toString())
+            .queryParam("include-resource-allocations", "true")
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    JSONObject reservations = json.getJSONObject("reservations");
+
+    testRDLHelper(reservations);
+
+    String reservationName = reservations.getJSONObject
+            ("reservation-definition").getString("reservation-name");
+    assertEquals(reservationName, "res_2");
+
+    rm.stop();
+  }
+
+  @Test
+  public void testInvalidStartTimeRequestListReservation() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    long time = clock.getTime() + MINIMUM_RESOURCE_DURATION;
+
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time, "res_1", 1);
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION,
+            "res_2", 2);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("start-time", "-1")
+            .queryParam("end-time", new Long((long)(time +
+                    MINIMUM_RESOURCE_DURATION * 0.9)).toString())
+            .queryParam("include-resource-allocations", "true")
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    JSONObject reservations = json.getJSONObject("reservations");
+
+    testRDLHelper(reservations);
+
+    // only res_1 should fall into the time interval given in the request json.
+    String reservationName = reservations.getJSONObject
+            ("reservation-definition").getString("reservation-name");
+    assertEquals(reservationName, "res_1");
+
+    rm.stop();
+  }
+
+  @Test
+  public void testEmptyStartTimeRequestListReservation() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    long time = clock.getTime() + MINIMUM_RESOURCE_DURATION;
+
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time, "res_1", 1);
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION,
+            "res_2", 2);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("end-time", new Long((long)(time +
+                    MINIMUM_RESOURCE_DURATION * 0.9)).toString())
+            .queryParam("include-resource-allocations", "true")
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    JSONObject reservations = json.getJSONObject("reservations");
+
+    testRDLHelper(reservations);
+
+    // only res_1 should fall into the time interval given in the request json.
+    String reservationName = reservations.getJSONObject
+            ("reservation-definition").getString("reservation-name");
+    assertEquals(reservationName, "res_1");
+
+    rm.stop();
+  }
+
+  @Test
+  public void testQueueOnlyRequestListReservation() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1);
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_2", 2);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    assertEquals(json.getJSONArray("reservations").length(), 2);
+
+    testRDLHelper(json.getJSONArray("reservations").getJSONObject(0));
+    testRDLHelper(json.getJSONArray("reservations").getJSONObject(1));
+
+    rm.stop();
+  }
+
+  @Test
+  public void testEmptyQueueRequestListReservation() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1);
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_2", 2);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH);
+
+    testListReservationHelper(resource, Status.BAD_REQUEST);
+
+    rm.stop();
+  }
+
+  @Test
+  public void testNonExistentQueueRequestListReservation() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1);
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_2", 2);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("queue", DEFAULT_QUEUE + "_invalid");
+
+    testListReservationHelper(resource, Status.BAD_REQUEST);
+
+    rm.stop();
+  }
+
+  @Test
+  public void testReservationIdRequestListReservation() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    ReservationId id1 = testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1);
+    testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_2", 2);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("include-resource-allocations", "true")
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    if (id1 != null) {
+      resource = resource.queryParam("reservation-id", id1.toString());
+    }
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    JSONObject reservations = json.getJSONObject("reservations");
+
+    testRDLHelper(reservations);
+
+    long reservationId = reservations.getJSONObject("reservation-id")
+            .getLong("reservation-id");
+    assertEquals(id1.getId(), reservationId);
+
+    rm.stop();
+  }
+
+  @Test
+  public void testInvalidReservationIdRequestListReservation() throws
+          Exception {
+    rm.start();
+    setupCluster(100);
+
+    ReservationId id1 = testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    if (id1 != null) {
+      resource = resource.queryParam("reservation-id",
+              "invalid" + id1.toString());
+    }
+
+    JSONObject response = testListReservationHelper(resource, Status.NOT_FOUND);
+
+    rm.stop();
+  }
+
+  @Test
+  public void testIncludeResourceAllocations() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    ReservationId id1 = testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("include-resource-allocations", "true")
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    if (id1 != null) {
+      resource = resource.queryParam("reservation-id", id1.toString());
+    }
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    JSONObject reservations = json.getJSONObject("reservations");
+
+    testRDLHelper(reservations);
+
+    long reservationId = reservations.getJSONObject("reservation-id")
+            .getLong("reservation-id");
+    assertEquals(id1.getId(), reservationId);
+
+    assertTrue(reservations.has("resource-allocations"));
+
+    rm.stop();
+  }
+
+  @Test
+  public void testExcludeResourceAllocations() throws Exception {
+    rm.start();
+    setupCluster(100);
+
+    ReservationId id1 = testSubmissionReservationHelper("reservation/submit",
+            MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1);
+
+    WebResource resource = constructWebResource(LIST_RESERVATION_PATH)
+            .queryParam("include-resource-allocations", "false")
+            .queryParam("queue", DEFAULT_QUEUE);
+
+    if (id1 != null) {
+      resource = resource.queryParam("reservation-id", id1.toString());
+    }
+
+    JSONObject json = testListReservationHelper(resource);
+
+    if (!this.isAuthenticationEnabled() && json == null) {
+      return;
+    }
+
+    JSONObject reservations = json.getJSONObject("reservations");
+
+    testRDLHelper(reservations);
+
+    long reservationId = reservations.getJSONObject("reservation-id")
+            .getLong("reservation-id");
+    assertEquals(id1.getId(), reservationId);
+
+    assertTrue(!reservations.has("resource-allocations"));
+
+    rm.stop();
+  }
+
+  @Test
   public void testDeleteReservation() throws JSONException, Exception {
     rm.start();
     for (int i = 0; i < 100; i++) {
@@ -391,10 +834,27 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
   }
 
   private ReservationId testSubmissionReservationHelper(String path,
-      String media) throws JSONException, Exception {
+      String media) throws Exception {
+    long arrival = clock.getTime() + MINIMUM_RESOURCE_DURATION;
+
+    return testSubmissionReservationHelper(path, media, arrival, "res_1", 1);
+  }
 
+  private ReservationId testSubmissionReservationHelper(String path,
+      String media, Long arrival, String reservationName, int expectedId)
+      throws Exception {
     String reservationJson = loadJsonFile("submit-reservation.json");
 
+    String reservationJsonRequest = String.format(reservationJson, arrival,
+            arrival + MINIMUM_RESOURCE_DURATION, reservationName);
+
+    return submitAndVerifyReservation(path, media, reservationJsonRequest,
+            expectedId);
+  }
+
+  private ReservationId submitAndVerifyReservation(String path, String media,
+      String reservationJson, int expectedId) throws Exception {
+
     JSONJAXBContext jc =
         new JSONJAXBContext(JSONConfiguration.mapped()
             .build(), ReservationSubmissionRequestInfo.class);
@@ -421,7 +881,7 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
     ReservationId rid = null;
     try {
       rid = ReservationId.parseReservationId(json.getString("reservation-id"));
-      assertEquals("incorrect return value", rid.getId(), 1);
+      assertEquals("incorrect return value", rid.getId(), expectedId);
     } catch (JSONException j) {
       // failure is possible and is checked outside
     }
@@ -511,7 +971,45 @@ public class TestRMWebServicesReservation extends JerseyTestBase {
     System.out.println("RESPONSE:" + response);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     assertEquals(Status.OK, response.getClientResponseStatus());
+  }
+
+  private void testRDLHelper(JSONObject json) throws JSONException {
+    JSONObject requests = json.getJSONObject("reservation-definition")
+            .getJSONObject("reservation-requests");
+    String type = requests.getString
+            ("reservation-request-interpreter");
+
+    assertEquals("0", type);
+    assertEquals(60, requests.getJSONArray("reservation-request")
+            .getJSONObject(0).getInt("duration"));
+  }
 
+  private JSONObject testListReservationHelper(WebResource resource) throws
+          Exception {
+    return testListReservationHelper(resource, Status.OK);
   }
 
+  private JSONObject testListReservationHelper(WebResource resource, Status
+          status) throws Exception {
+    Thread.sleep(1000);
+    ClientResponse response = resource.get(ClientResponse.class);
+
+    if (!this.isAuthenticationEnabled()) {
+      assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+      return null;
+    }
+
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(status, response.getClientResponseStatus());
+
+    return response.getEntity(JSONObject.class);
+  }
+
+  private void setupCluster(int nodes) throws Exception {
+    for (int i = 0; i < nodes; i++) {
+      MockNM amNodeManager =
+              rm.registerNode("127.0.0." + i + ":1234", 100 * 1024);
+      amNodeManager.nodeHeartbeat(true);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b706cbc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json
index 573d317..0a243a2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json
@@ -1,9 +1,9 @@
 {
   "queue" : "dedicated",
   "reservation-definition" : {
-     "arrival" : 1765541532000,
-     "deadline" : 1765542252000,
-     "reservation-name" : "res_1",
+     "arrival" : %s,
+     "deadline" : %s,
+     "reservation-name" : "%s",
      "reservation-requests" : {
         "reservation-request-interpreter" : 0,
         "reservation-request" : [


[2/2] hadoop git commit: Merge branch 'trunk' of https://git-wip-us.apache.org/repos/asf/hadoop into trunk

Posted by cu...@apache.org.
Merge branch 'trunk' of https://git-wip-us.apache.org/repos/asf/hadoop into trunk


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c3641ed3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c3641ed3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c3641ed3

Branch: refs/heads/trunk
Commit: c3641ed3b183fa46e6b042fd19c1a373b8f95d33
Parents: b706cbc bc425a6
Author: = <cu...@apache.org>
Authored: Wed Feb 10 10:24:18 2016 -0800
Committer: = <cu...@apache.org>
Committed: Wed Feb 10 10:24:18 2016 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../sink/TestRollingFileSystemSinkWithHdfs.java | 287 +++++++++++++++++++
 2 files changed, 289 insertions(+)
----------------------------------------------------------------------