Browse Source

YARN-1708. Public YARN APIs for creating/updating/deleting reservations.

subru 10 years ago
parent
commit
3f2e3b275b
32 changed files with 2848 additions and 6 deletions
  1. 2 0
      YARN-1051-CHANGES.txt
  2. 113 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
  3. 72 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteRequest.java
  4. 48 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java
  5. 97 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java
  6. 78 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java
  7. 96 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateRequest.java
  8. 48 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java
  9. 22 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
  10. 137 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
  11. 147 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
  12. 222 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
  13. 101 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
  14. 94 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java
  15. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
  16. 36 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  17. 28 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  18. 57 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
  19. 61 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
  20. 127 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteRequestPBImpl.java
  21. 68 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteResponsePBImpl.java
  22. 152 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionRequestPBImpl.java
  23. 129 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionResponsePBImpl.java
  24. 169 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateRequestPBImpl.java
  25. 68 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateResponsePBImpl.java
  26. 32 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
  27. 17 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
  28. 169 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
  29. 75 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationIdPBImpl.java
  30. 152 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestPBImpl.java
  31. 189 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java
  32. 39 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/UTCClock.java

+ 2 - 0
YARN-1051-CHANGES.txt

@@ -3,3 +3,5 @@ CapacityScheduler. (Carlo Curino and Subru Krishnan via curino)
 
 YARN-2475. Logic for responding to capacity drops for the 
 ReservationSystem. (Carlo Curino and Subru Krishnan via curino)
+
+YARN-1708. Public YARN APIs for creating/updating/deleting reservations. (subru)

+ 113 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java

@@ -57,6 +57,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesReq
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -67,6 +73,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.Token;
@@ -543,4 +550,110 @@ public interface ApplicationClientProtocol {
   public GetContainersResponse getContainers(GetContainersRequest request)
       throws YarnException, IOException;
 
+  /**
+   * <p>
+   * The interface used by clients to submit a new reservation to the
+   * {@link ResourceManager}.
+   * </p>
+   * 
+   * <p>
+   * The client packages all details of its request in a
+   * {@link ReservationSubmissionRequest} object. This contains information
+   * about the amount of capacity, temporal constraints, and concurrency needs.
+   * Furthermore, the reservation might be composed of multiple stages, with
+   * ordering dependencies among them.
+   * </p>
+   * 
+   * <p>
+   * In order to respond, a new admission control component in the
+   * {@link ResourceManager} performs an analysis of the resources that have
+   * been committed over the period of time the user is requesting, verify that
+   * the user requests can be fulfilled, and that it respect a sharing policy
+   * (e.g., {@link CapacityOverTimePolicy}). Once it has positively determined
+   * that the ReservationSubmissionRequest is satisfiable the
+   * {@link ResourceManager} answers with a
+   * {@link ReservationSubmissionResponse} that include a non-null
+   * {@link ReservationId}. Upon failure to find a valid allocation the response
+   * is an exception with the reason.
+   * 
+   * On application submission the client can use this {@link ReservationId} to
+   * obtain access to the reserved resources.
+   * </p>
+   * 
+   * <p>
+   * The system guarantees that during the time-range specified by the user, the
+   * reservationID will be corresponding to a valid reservation. The amount of
+   * capacity dedicated to such queue can vary overtime, depending of the
+   * allocation that has been determined. But it is guaranteed to satisfy all
+   * the constraint expressed by the user in the
+   * {@link ReservationSubmissionRequest}.
+   * </p>
+   * 
+   * @param request the request to submit a new Reservation
+   * @return response the {@link ReservationId} on accepting the submission
+   * @throws YarnException if the request is invalid or reservation cannot be
+   *           created successfully
+   * @throws IOException
+   * 
+   */
+  @Public
+  @Unstable
+  public ReservationSubmissionResponse submitReservation(
+      ReservationSubmissionRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to update an existing Reservation. This is
+   * referred to as a re-negotiation process, in which a user that has
+   * previously submitted a Reservation.
+   * </p>
+   * 
+   * <p>
+   * The allocation is attempted by virtually substituting all previous
+   * allocations related to this Reservation with new ones, that satisfy the new
+   * {@link ReservationUpdateRequest}. Upon success the previous allocation is
+   * substituted by the new one, and on failure (i.e., if the system cannot find
+   * a valid allocation for the updated request), the previous allocation
+   * remains valid.
+   * 
+   * The {@link ReservationId} is not changed, and applications currently
+   * running within this reservation will automatically receive the resources
+   * based on the new allocation.
+   * </p>
+   * 
+   * @param request to update an existing Reservation (the ReservationRequest
+   *          should refer to an existing valid {@link ReservationId})
+   * @return response empty on successfully updating the existing reservation
+   * @throws YarnException if the request is invalid or reservation cannot be
+   *           updated successfully
+   * @throws IOException
+   * 
+   */
+  @Public
+  @Unstable
+  public ReservationUpdateResponse updateReservation(
+      ReservationUpdateRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to remove an existing Reservation.
+   * 
+   * Upon deletion of a reservation applications running with this reservation,
+   * are automatically downgraded to normal jobs running without any dedicated
+   * reservation.
+   * </p>
+   * 
+   * @param request to remove an existing Reservation (the ReservationRequest
+   *          should refer to an existing valid {@link ReservationId})
+   * @return response empty on successfully deleting the existing reservation
+   * @throws YarnException if the request is invalid or reservation cannot be
+   *           deleted successfully
+   * @throws IOException
+   * 
+   */
+  @Public
+  @Unstable
+  public ReservationDeleteResponse deleteReservation(
+      ReservationDeleteRequest request) throws YarnException, IOException;
+
 }

+ 72 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteRequest.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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationDeleteRequest} captures the set of requirements the user
+ * has to delete an existing reservation.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationDeleteRequest {
+
+  @Public
+  @Unstable
+  public static ReservationDeleteRequest newInstance(ReservationId reservationId) {
+    ReservationDeleteRequest request =
+        Records.newRecord(ReservationDeleteRequest.class);
+    request.setReservationId(reservationId);
+    return request;
+  }
+
+  /**
+   * Get the {@link ReservationId}, that corresponds to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @return the {@link ReservationId} representing the unique id of the
+   *         corresponding reserved resource allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract ReservationId getReservationId();
+
+  /**
+   * Set the {@link ReservationId}, that correspond to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @param reservationId the {@link ReservationId} representing the the unique
+   *          id of the corresponding reserved resource allocation in the
+   *          scheduler
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationId(ReservationId reservationId);
+
+}

+ 48 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationDeleteResponse.java

@@ -0,0 +1,48 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationDeleteResponse} contains the answer of the admission
+ * control system in the {@link ResourceManager} to a reservation delete
+ * operation. Currently response is empty if the operation was successful, if
+ * not an exception reporting reason for a failure.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationDeleteResponse {
+
+  @Private
+  @Unstable
+  public static ReservationDeleteResponse newInstance() {
+    ReservationDeleteResponse response =
+        Records.newRecord(ReservationDeleteResponse.class);
+    return response;
+  }
+
+}

+ 97 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java

@@ -0,0 +1,97 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationSubmissionRequest} captures the set of requirements the
+ * user has to create a reservation.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationSubmissionRequest {
+
+  @Public
+  @Unstable
+  public static ReservationSubmissionRequest newInstance(
+      ReservationDefinition reservationDefinition, String queueName) {
+    ReservationSubmissionRequest request =
+        Records.newRecord(ReservationSubmissionRequest.class);
+    request.setReservationDefinition(reservationDefinition);
+    request.setQueue(queueName);
+    return request;
+  }
+
+  /**
+   * Get the {@link ReservationDefinition} representing the user constraints for
+   * this reservation
+   * 
+   * @return the reservation definition representing user constraints
+   */
+  @Public
+  @Unstable
+  public abstract ReservationDefinition getReservationDefinition();
+
+  /**
+   * Set the {@link ReservationDefinition} representing the user constraints for
+   * this reservation
+   * 
+   * @param reservationDefinition the reservation request representing the
+   *          reservation
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationDefinition(
+      ReservationDefinition reservationDefinition);
+
+  /**
+   * Get the name of the {@link Plan} that corresponds to the name of the
+   * {@link QueueInfo} in the scheduler to which the reservation will be
+   * submitted to.
+   * 
+   * @return the name of the {@link Plan} that corresponds to the name of the
+   *         {@link QueueInfo} in the scheduler to which the reservation will be
+   *         submitted to
+   */
+  @Public
+  @Unstable
+  public abstract String getQueue();
+
+  /**
+   * Set the name of the {@link Plan} that corresponds to the name of the
+   * {@link QueueInfo} in the scheduler to which the reservation will be
+   * submitted to
+   * 
+   * @param the name of the parent {@link Plan} that corresponds to the name of
+   *          the {@link QueueInfo} in the scheduler to which the reservation
+   *          will be submitted to
+   */
+  @Public
+  @Unstable
+  public abstract void setQueue(String queueName);
+
+}

+ 78 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java

@@ -0,0 +1,78 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.util.Records;
+
+import com.sun.naming.internal.ResourceManager;
+
+/**
+ * {@link ReservationSubmissionResponse} contains the answer of the admission
+ * control system in the {@link ResourceManager} to a reservation create
+ * operation. Response contains a {@link ReservationId} if the operation was
+ * successful, if not an exception reporting reason for a failure.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationSubmissionResponse {
+
+  @Private
+  @Unstable
+  public static ReservationSubmissionResponse newInstance(
+      ReservationId reservationId) {
+    ReservationSubmissionResponse response =
+        Records.newRecord(ReservationSubmissionResponse.class);
+    response.setReservationId(reservationId);
+    return response;
+  }
+
+  /**
+   * Get the {@link ReservationId}, that corresponds to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @return the {@link ReservationId} representing the unique id of the
+   *         corresponding reserved resource allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract ReservationId getReservationId();
+
+  /**
+   * Set the {@link ReservationId}, that correspond to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @param reservationId the {@link ReservationId} representing the the unique
+   *          id of the corresponding reserved resource allocation in the
+   *          scheduler
+   */
+  @Private
+  @Unstable
+  public abstract void setReservationId(ReservationId reservationId);
+
+}

+ 96 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateRequest.java

@@ -0,0 +1,96 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationUpdateRequest} captures the set of requirements the user
+ * has to update an existing reservation.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationUpdateRequest {
+
+  @Public
+  @Unstable
+  public static ReservationUpdateRequest newInstance(
+      ReservationDefinition reservationDefinition, ReservationId reservationId) {
+    ReservationUpdateRequest request =
+        Records.newRecord(ReservationUpdateRequest.class);
+    request.setReservationDefinition(reservationDefinition);
+    request.setReservationId(reservationId);
+    return request;
+  }
+
+  /**
+   * Get the {@link ReservationDefinition} representing the updated user
+   * constraints for this reservation
+   * 
+   * @return the reservation definition representing user constraints
+   */
+  @Public
+  @Unstable
+  public abstract ReservationDefinition getReservationDefinition();
+
+  /**
+   * Set the {@link ReservationDefinition} representing the updated user
+   * constraints for this reservation
+   * 
+   * @param reservationDefinition the reservation request representing the
+   *          reservation
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationDefinition(
+      ReservationDefinition reservationDefinition);
+
+  /**
+   * Get the {@link ReservationId}, that corresponds to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @return the {@link ReservationId} representing the unique id of the
+   *         corresponding reserved resource allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract ReservationId getReservationId();
+
+  /**
+   * Set the {@link ReservationId}, that correspond to a valid resource
+   * allocation in the scheduler (between start and end time of this
+   * reservation)
+   * 
+   * @param reservationId the {@link ReservationId} representing the the unique
+   *          id of the corresponding reserved resource allocation in the
+   *          scheduler
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationId(ReservationId reservationId);
+
+}

+ 48 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationUpdateResponse.java

@@ -0,0 +1,48 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationUpdateResponse} contains the answer of the admission
+ * control system in the {@link ResourceManager} to a reservation update
+ * operation. Currently response is empty if the operation was successful, if
+ * not an exception reporting reason for a failure.
+ * 
+ * @see ReservationDefinition
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationUpdateResponse {
+
+  @Private
+  @Unstable
+  public static ReservationUpdateResponse newInstance() {
+    ReservationUpdateResponse response =
+        Records.newRecord(ReservationUpdateResponse.class);
+    return response;
+  }
+
+}

+ 22 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java

@@ -417,4 +417,26 @@ public abstract class ApplicationSubmissionContext {
   @Stable
   public abstract void setLogAggregationContext(
       LogAggregationContext logAggregationContext);
+
+  /**
+   * Get the reservation id, that corresponds to a valid resource allocation in
+   * the scheduler (between start and end time of the corresponding reservation)
+   * 
+   * @return the reservation id representing the unique id of the corresponding
+   *         reserved resource allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract ReservationId getReservationID();
+
+  /**
+   * Set the reservation id, that correspond to a valid resource allocation in
+   * the scheduler (between start and end time of the corresponding reservation)
+   * 
+   * @param reservationId representing the unique id of the
+   *          corresponding reserved resource allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationID(ReservationId reservationID);
 }

+ 137 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java

@@ -0,0 +1,137 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationDefinition} captures the set of resource and time
+ * constraints the user cares about regarding a reservation.
+ * 
+ * @see ResourceRequest
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationDefinition {
+
+  @Public
+  @Unstable
+  public static ReservationDefinition newInstance(long arrival, long deadline,
+      ReservationRequests reservationRequests, String name) {
+    ReservationDefinition rDefinition =
+        Records.newRecord(ReservationDefinition.class);
+    rDefinition.setArrival(arrival);
+    rDefinition.setDeadline(deadline);
+    rDefinition.setReservationRequests(reservationRequests);
+    rDefinition.setReservationName(name);
+    return rDefinition;
+  }
+
+  /**
+   * Get the arrival time or the earliest time from which the resource(s) can be
+   * allocated. Time expressed as UTC.
+   * 
+   * @return the earliest valid time for this reservation
+   */
+  @Public
+  @Unstable
+  public abstract long getArrival();
+
+  /**
+   * Set the arrival time or the earliest time from which the resource(s) can be
+   * allocated. Time expressed as UTC.
+   * 
+   * @param earliestStartTime the earliest valid time for this reservation
+   */
+  @Public
+  @Unstable
+  public abstract void setArrival(long earliestStartTime);
+
+  /**
+   * Get the deadline or the latest time by when the resource(s) must be
+   * allocated. Time expressed as UTC.
+   * 
+   * @return the deadline or the latest time by when the resource(s) must be
+   *         allocated
+   */
+  @Public
+  @Unstable
+  public abstract long getDeadline();
+
+  /**
+   * Set the deadline or the latest time by when the resource(s) must be
+   * allocated. Time expressed as UTC.
+   * 
+   * @param latestEndTime the deadline or the latest time by when the
+   *          resource(s) should be allocated
+   */
+  @Public
+  @Unstable
+  public abstract void setDeadline(long latestEndTime);
+
+  /**
+   * Get the list of {@link ReservationRequests} representing the resources
+   * required by the application
+   * 
+   * @return the list of {@link ReservationRequests}
+   */
+  @Public
+  @Unstable
+  public abstract ReservationRequests getReservationRequests();
+
+  /**
+   * Set the list of {@link ReservationRequests} representing the resources
+   * required by the application
+   * 
+   * @param resources the list of {@link ReservationRequests}
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationRequests(
+      ReservationRequests reservationRequests);
+
+  /**
+   * Get the name for this reservation. The name need-not be unique, and it is
+   * just a mnemonic for the user (akin to job names). Accepted reservations are
+   * uniquely identified by a system-generated ReservationId.
+   * 
+   * @return string representing the name of the corresponding reserved resource
+   *         allocation in the scheduler
+   */
+  @Public
+  @Evolving
+  public abstract String getReservationName();
+
+  /**
+   * Set the name for this reservation. The name need-not be unique, and it is
+   * just a mnemonic for the user (akin to job names). Accepted reservations are
+   * uniquely identified by a system-generated ReservationId.
+   * 
+   * @param name representing the name of the corresponding reserved resource
+   *          allocation in the scheduler
+   */
+  @Public
+  @Evolving
+  public abstract void setReservationName(String name);
+
+}

+ 147 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java

@@ -0,0 +1,147 @@
+/**
+ * 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.api.records;
+
+import java.text.NumberFormat;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * {@link ReservationId} represents the <em>globally unique</em> identifier for
+ * a reservation.
+ * </p>
+ * 
+ * <p>
+ * The globally unique nature of the identifier is achieved by using the
+ * <em>cluster timestamp</em> i.e. start-time of the {@link ResourceManager}
+ * along with a monotonically increasing counter for the reservation.
+ * </p>
+ */
+@Public
+@Unstable
+public abstract class ReservationId implements Comparable<ReservationId> {
+
+  @Private
+  @Unstable
+  public static final String reserveIdStrPrefix = "reservation_";
+  protected long clusterTimestamp;
+  protected long id;
+
+  @Private
+  @Unstable
+  public static ReservationId newInstance(long clusterTimestamp, long id) {
+    ReservationId reservationId = Records.newRecord(ReservationId.class);
+    reservationId.setClusterTimestamp(clusterTimestamp);
+    reservationId.setId(id);
+    reservationId.build();
+    return reservationId;
+  }
+
+  /**
+   * Get the long identifier of the {@link ReservationId} which is unique for
+   * all Reservations started by a particular instance of the
+   * {@link ResourceManager}.
+   * 
+   * @return long identifier of the {@link ReservationId}
+   */
+  @Public
+  @Unstable
+  public abstract long getId();
+
+  @Private
+  @Unstable
+  protected abstract void setId(long id);
+
+  /**
+   * Get the <em>start time</em> of the {@link ResourceManager} which is used to
+   * generate globally unique {@link ReservationId}.
+   * 
+   * @return <em>start time</em> of the {@link ResourceManager}
+   */
+  @Public
+  @Unstable
+  public abstract long getClusterTimestamp();
+
+  @Private
+  @Unstable
+  protected abstract void setClusterTimestamp(long clusterTimestamp);
+
+  protected abstract void build();
+
+  static final ThreadLocal<NumberFormat> reservIdFormat =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(4);
+          return fmt;
+        }
+      };
+
+  @Override
+  public int compareTo(ReservationId other) {
+    if (this.getClusterTimestamp() - other.getClusterTimestamp() == 0) {
+      return getId() > getId() ? 1 : getId() < getId() ? -1 : 0;
+    } else {
+      return this.getClusterTimestamp() > other.getClusterTimestamp() ? 1
+          : this.getClusterTimestamp() < other.getClusterTimestamp() ? -1 : 0;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return reserveIdStrPrefix + this.getClusterTimestamp() + "_"
+        + reservIdFormat.get().format(getId());
+  }
+
+  @Override
+  public int hashCode() {
+    // generated by eclipse
+    final int prime = 31;
+    int result = 1;
+    result =
+        prime * result
+            + (int) (getClusterTimestamp() ^ (getClusterTimestamp() >>> 32));
+    result = prime * result + (int) (getId() ^ (getId() >>> 32));
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    // generated by eclipse
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ReservationId other = (ReservationId) obj;
+    if (getClusterTimestamp() != other.getClusterTimestamp())
+      return false;
+    if (getId() != other.getId())
+      return false;
+    return true;
+  }
+
+}

+ 222 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java

@@ -0,0 +1,222 @@
+/**
+ * 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.api.records;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * {@link ReservationRequest} represents the request made by an application to
+ * the {@link ResourceManager} to reserve {@link Resource}s.
+ * </p>
+ * 
+ * <p>
+ * It includes:
+ * <ul>
+ * <li>{@link Resource} required for each request.</li>
+ * <li>
+ * Number of containers, of above specifications, which are required by the
+ * application.</li>
+ * <li>
+ * Concurrency that indicates the gang size of the request.</li>
+ * </ul>
+ * </p>
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationRequest implements
+    Comparable<ReservationRequest> {
+
+  @Public
+  @Unstable
+  public static ReservationRequest newInstance(Resource capability,
+      int numContainers) {
+    return newInstance(capability, numContainers, 1, -1);
+  }
+
+  @Public
+  @Unstable
+  public static ReservationRequest newInstance(Resource capability,
+      int numContainers, int concurrency, long duration) {
+    ReservationRequest request = Records.newRecord(ReservationRequest.class);
+    request.setCapability(capability);
+    request.setNumContainers(numContainers);
+    request.setConcurrency(concurrency);
+    request.setDuration(duration);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public static class ReservationRequestComparator implements
+      java.util.Comparator<ReservationRequest>, Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public int compare(ReservationRequest r1, ReservationRequest r2) {
+      // Compare numContainers, concurrency and capability
+      int ret = r1.getNumContainers() - r2.getNumContainers();
+      if (ret == 0) {
+        ret = r1.getConcurrency() - r2.getConcurrency();
+      }
+      if (ret == 0) {
+        ret = r1.getCapability().compareTo(r2.getCapability());
+      }
+      return ret;
+    }
+  }
+
+  /**
+   * Get the {@link Resource} capability of the request.
+   * 
+   * @return {@link Resource} capability of the request
+   */
+  @Public
+  @Unstable
+  public abstract Resource getCapability();
+
+  /**
+   * Set the {@link Resource} capability of the request
+   * 
+   * @param capability {@link Resource} capability of the request
+   */
+  @Public
+  @Unstable
+  public abstract void setCapability(Resource capability);
+
+  /**
+   * Get the number of containers required with the given specifications.
+   * 
+   * @return number of containers required with the given specifications
+   */
+  @Public
+  @Unstable
+  public abstract int getNumContainers();
+
+  /**
+   * Set the number of containers required with the given specifications
+   * 
+   * @param numContainers number of containers required with the given
+   *          specifications
+   */
+  @Public
+  @Unstable
+  public abstract void setNumContainers(int numContainers);
+
+  /**
+   * Get the number of containers that need to be scheduled concurrently. The
+   * default value of 1 would fall back to the current non concurrency
+   * constraints on the scheduling behavior.
+   * 
+   * @return the number of containers to be concurrently scheduled
+   */
+  @Public
+  @Unstable
+  public abstract int getConcurrency();
+
+  /**
+   * Set the number of containers that need to be scheduled concurrently. The
+   * default value of 1 would fall back to the current non concurrency
+   * constraints on the scheduling behavior.
+   * 
+   * @param numContainers the number of containers to be concurrently scheduled
+   */
+  @Public
+  @Unstable
+  public abstract void setConcurrency(int numContainers);
+
+  /**
+   * Get the duration in milliseconds for which the resource is required. A
+   * default value of -1, indicates an unspecified lease duration, and fallback
+   * to current behavior.
+   * 
+   * @return the duration in milliseconds for which the resource is required
+   */
+  @Public
+  @Unstable
+  public abstract long getDuration();
+
+  /**
+   * Set the duration in milliseconds for which the resource is required.
+   * 
+   * @param duration the duration in milliseconds for which the resource is
+   *          required
+   */
+  @Public
+  @Unstable
+  public abstract void setDuration(long duration);
+
+  @Override
+  public int hashCode() {
+    final int prime = 2153;
+    int result = 2459;
+    Resource capability = getCapability();
+    result =
+        prime * result + ((capability == null) ? 0 : capability.hashCode());
+    result = prime * result + getNumContainers();
+    result = prime * result + getConcurrency();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ReservationRequest other = (ReservationRequest) obj;
+    Resource capability = getCapability();
+    if (capability == null) {
+      if (other.getCapability() != null)
+        return false;
+    } else if (!capability.equals(other.getCapability()))
+      return false;
+    if (getNumContainers() != other.getNumContainers())
+      return false;
+    if (getConcurrency() != other.getConcurrency())
+      return false;
+    return true;
+  }
+
+  @Override
+  public int compareTo(ReservationRequest other) {
+    int numContainersComparison =
+        this.getNumContainers() - other.getNumContainers();
+    if (numContainersComparison == 0) {
+      int concurrencyComparison =
+          this.getConcurrency() - other.getConcurrency();
+      if (concurrencyComparison == 0) {
+        return this.getCapability().compareTo(other.getCapability());
+      } else {
+        return concurrencyComparison;
+      }
+    } else {
+      return numContainersComparison;
+    }
+  }
+
+}

+ 101 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java

@@ -0,0 +1,101 @@
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Enumeration of various types of dependencies among multiple
+ * {@link ReservationRequests} within one {@link ReservationDefinition} (from
+ * least constraining to most constraining).
+ */
+@Public
+@Evolving
+public enum ReservationRequestInterpreter {
+  /**
+   * Requires that exactly ONE among the {@link ReservationRequest} submitted as
+   * of a {@link ReservationDefinition} is satisfied to satisfy the overall
+   * {@link ReservationDefinition}.
+   * 
+   * WHEN TO USE THIS: This is useful when the user have multiple equivalent
+   * ways to run an application, and wants to expose to the ReservationAgent
+   * such flexibility. For example an application could use one <32GB,16core>
+   * container for 10min, or 16 <2GB,1core> containers for 15min, the
+   * ReservationAgent will decide which one of the two it is best for the system
+   * to place.
+   * 
+   */
+  R_ANY,
+
+  /**
+   * Requires that ALL of the {@link ReservationRequest} submitted as part of a
+   * {@link ReservationDefinition} are satisfied for the overall
+   * {@link ReservationDefinition} to be satisfied. No constraints are imposed
+   * on the temporal ordering of the allocation used to satisfy the
+   * ResourceRequeusts.
+   * 
+   * WHEN TO USE THIS: This is useful to capture a scenario in which the user
+   * cares for multiple ReservationDefinition to be all accepted, or none. For
+   * example, a user might want a reservation R1: with 10 x <8GB,4core> for
+   * 10min, and a reservation R2: with 2 <1GB,1core> for 1h, and only if both
+   * are satisfied the workflow run in this reservation succeeds. The key
+   * differentiator from ALL and ORDER, ORDER_NO_GAP, is that ALL imposes no
+   * restrictions on the relative allocations used to place R1 and R2 above.
+   * 
+   */
+  R_ALL,
+
+  /**
+   * Requires that ALL of the {@link ReservationRequest} submitted as part of a
+   * {@link ReservationDefinition} are satisfied for the overall
+   * {@link ReservationDefinition} to be satisfied. Moreover, it imposes a
+   * strict temporal ordering on the allocation used to satisfy the
+   * {@link ReservationRequest}s. The allocations satisfying the
+   * {@link ReservationRequest} in position k must strictly precede the
+   * allocations for the {@link ReservationRequest} at position k+1. No
+   * constraints are imposed on temporal gaps between subsequent allocations
+   * (the last instant of the previous allocation can be an arbitrary long
+   * period of time before the first instant of the subsequent allocation).
+   * 
+   * WHEN TO USE THIS: Like ALL this requires all ReservationDefinitions to be
+   * placed, but it also imposes a time ordering on the allocations used. This
+   * is important if the ReservationDefinition(s) are used to describe a
+   * workflow with inherent inter-stage dependencies. For example, a first job
+   * runs in a ReservaitonDefinition R1 (10 x <1GB,1core> for 20min), and its
+   * output is consumed by a second job described by a ReservationDefinition R2
+   * (5 x <1GB,1core>) for 50min). R2 allocation cannot overlap R1, as R2 models
+   * a job depending on the output of the job modeled by R1.
+   */
+  R_ORDER,
+
+  /**
+   * Requires that ALL of the {@link ReservationRequest} submitted as part of a
+   * {@link ReservationDefinition} are satisfied for the overall
+   * {@link ReservationDefinition} to be satisfied. Moreover, it imposes a
+   * strict temporal ordering on the allocation used to satisfy the
+   * {@link ResourceRequeust}s. It imposes a strict temporal ordering on the
+   * allocation used to satisfy the {@link ReservationRequest}s. The allocations
+   * satisfying the {@link ReservationRequest} in position k must strictly
+   * precede the allocations for the {@link ReservationRequest} at position k+1.
+   * Moreover it imposes a "zero-size gap" between subsequent allocations, i.e.,
+   * the last instant in time of the allocations associated with the
+   * {@link ReservationRequest} at position k must be exactly preceding the
+   * first instant in time of the {@link ReservationRequest} at position k+1.
+   * Time ranges are interpreted as [a,b) inclusive left, exclusive right.
+   * 
+   * WHEN TO USE THIS: This is a stricter version of R_ORDER, which allows no
+   * gaps between the allocations that satisfy R1 and R2. The use of this is
+   * twofold: 1) prevent long gaps between subsequent stages that produce very
+   * large intermediate output (e.g., the output of R1 is too large to be kept
+   * around for long before the job running in R2 consumes it, and disposes of
+   * it), 2) if the job being modeled has a time-varying resource need, one can
+   * combine multiple ResourceDefinition each approximating a portion of the job
+   * execution (think of using multiple rectangular bounding boxes to described
+   * an arbitrarily shaped area). By asking for no-gaps we guarantee
+   * "continuity" of resources given to this job. This still allow for some
+   * flexibility, as the entire "train" of allocations can be moved rigidly back
+   * or forth within the start-deadline time range (if there is slack).
+   * 
+   */
+  R_ORDER_NO_GAP
+
+}

+ 94 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequests.java

@@ -0,0 +1,94 @@
+/**
+ * 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.api.records;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@link ReservationRequests} captures the set of resource and constraints the
+ * user cares about regarding a reservation.
+ * 
+ * @see ReservationRequest
+ * 
+ */
+@Public
+@Unstable
+public abstract class ReservationRequests {
+
+  @Public
+  @Unstable
+  public static ReservationRequests newInstance(
+      List<ReservationRequest> reservationResources,
+      ReservationRequestInterpreter type) {
+    ReservationRequests reservationRequests =
+        Records.newRecord(ReservationRequests.class);
+    reservationRequests.setReservationResources(reservationResources);
+    reservationRequests.setInterpreter(type);
+    return reservationRequests;
+  }
+
+  /**
+   * Get the list of {@link ReservationRequest} representing the resources
+   * required by the application
+   * 
+   * @return the list of {@link ReservationRequest}
+   */
+  @Public
+  @Unstable
+  public abstract List<ReservationRequest> getReservationResources();
+
+  /**
+   * Set the list of {@link ReservationRequest} representing the resources
+   * required by the application
+   * 
+   * @param resources the list of {@link ReservationRequest}
+   */
+  @Public
+  @Unstable
+  public abstract void setReservationResources(
+      List<ReservationRequest> reservationResources);
+
+  /**
+   * Get the {@link ReservationRequestInterpreter}, representing how the list of
+   * resources should be allocated, this captures temporal ordering and other
+   * constraints.
+   * 
+   * @return the list of {@link ReservationRequestInterpreter}
+   */
+  @Public
+  @Unstable
+  public abstract ReservationRequestInterpreter getInterpreter();
+
+  /**
+   * Set the {@link ReservationRequestInterpreter}, representing how the list of
+   * resources should be allocated, this captures temporal ordering and other
+   * constraints.
+   * 
+   * @param interpreter the {@link ReservationRequestInterpreter} for this
+   *          reservation
+   */
+  @Public
+  @Unstable
+  public abstract void setInterpreter(ReservationRequestInterpreter interpreter);
+
+}

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto

@@ -49,5 +49,8 @@ service ApplicationClientProtocolService {
   rpc getApplicationAttempts (GetApplicationAttemptsRequestProto) returns (GetApplicationAttemptsResponseProto);
   rpc getContainerReport (GetContainerReportRequestProto) returns (GetContainerReportResponseProto);
   rpc getContainers (GetContainersRequestProto) returns (GetContainersResponseProto);
+  rpc submitReservation (ReservationSubmissionRequestProto) returns (ReservationSubmissionResponseProto);
+  rpc updateReservation (ReservationUpdateRequestProto) returns (ReservationUpdateResponseProto);
+  rpc deleteReservation (ReservationDeleteRequestProto) returns (ReservationDeleteResponseProto);
 }
 

+ 36 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -202,7 +202,6 @@ message ApplicationAttemptReportProto {
   optional string diagnostics = 5 [default = "N/A"];
   optional YarnApplicationAttemptStateProto yarn_application_attempt_state = 6;
   optional ContainerIdProto am_container_id = 7;
-  optional string original_tracking_url = 8;
 }
 
 enum NodeStateProto {
@@ -293,6 +292,7 @@ message ApplicationSubmissionContextProto {
   repeated string applicationTags = 12;
   optional int64 attempt_failures_validity_interval = 13 [default = -1];
   optional LogAggregationContextProto log_aggregation_context = 14;
+  optional ReservationIdProto reservation_id = 15;
 }
 
 message LogAggregationContextProto {
@@ -340,6 +340,41 @@ message QueueUserACLInfoProto {
   repeated QueueACLProto userAcls = 2;
 }
 
+////////////////////////////////////////////////////////////////////////
+////// From reservation_protocol /////////////////////////////////////
+////////////////////////////////////////////////////////////////////////
+
+message ReservationIdProto {
+  optional int64 id = 1;
+  optional int64 cluster_timestamp = 2;
+}
+
+message ReservationRequestProto {
+  optional ResourceProto capability = 1;
+  optional int32 num_containers = 2 [default = 1];
+  optional int32 concurrency = 3 [default = 1];
+  optional int64 duration = 4 [default = -1];
+}
+
+message ReservationRequestsProto {
+  repeated ReservationRequestProto reservation_resources = 1;
+  optional ReservationRequestInterpreterProto interpreter = 2 [default = R_ALL];
+}
+
+message ReservationDefinitionProto {
+  optional ReservationRequestsProto reservation_requests = 1;
+  optional int64 arrival = 2;
+  optional int64 deadline = 3;
+  optional string reservation_name = 4;
+}
+
+enum ReservationRequestInterpreterProto {
+  R_ANY = 0;
+  R_ALL = 1;
+  R_ORDER = 2;
+  R_ORDER_NO_GAP = 3;
+ }
+
 ////////////////////////////////////////////////////////////////////////
 ////// From container_manager //////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////

+ 28 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto

@@ -292,3 +292,31 @@ message GetContainersRequestProto {
 message GetContainersResponseProto {
   repeated ContainerReportProto containers = 1;
 }
+
+//////////////////////////////////////////////////////
+//  reservation_protocol
+//////////////////////////////////////////////////////
+
+message ReservationSubmissionRequestProto {
+  optional string queue = 1;
+  optional ReservationDefinitionProto reservation_definition = 2;
+}
+
+message ReservationSubmissionResponseProto {
+    optional ReservationIdProto reservation_id = 1;
+}
+
+message ReservationUpdateRequestProto {
+  optional ReservationDefinitionProto reservation_definition = 1;
+  optional ReservationIdProto reservation_id = 2;
+}
+
+message ReservationUpdateResponseProto {   
+}
+
+message ReservationDeleteRequestProto {
+  optional ReservationIdProto reservation_id = 1;
+}
+
+message ReservationDeleteResponseProto {
+}

+ 57 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java

@@ -63,6 +63,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesReq
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl;
@@ -97,6 +103,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQ
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQueuesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -110,13 +122,15 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersRequestProto;
 
-
 import com.google.protobuf.ServiceException;
 
 @Private
@@ -391,4 +405,46 @@ public class ApplicationClientProtocolPBClientImpl implements ApplicationClientP
     }
   }
 
+  @Override
+  public ReservationSubmissionResponse submitReservation(ReservationSubmissionRequest request)
+      throws YarnException, IOException {
+    ReservationSubmissionRequestProto requestProto =
+        ((ReservationSubmissionRequestPBImpl) request).getProto();
+    try {
+      return new ReservationSubmissionResponsePBImpl(proxy.submitReservation(null,
+          requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
+
+  @Override
+  public ReservationUpdateResponse updateReservation(ReservationUpdateRequest request)
+      throws YarnException, IOException {
+    ReservationUpdateRequestProto requestProto =
+        ((ReservationUpdateRequestPBImpl) request).getProto();
+    try {
+      return new ReservationUpdateResponsePBImpl(proxy.updateReservation(null,
+          requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
+
+  @Override
+  public ReservationDeleteResponse deleteReservation(ReservationDeleteRequest request)
+      throws YarnException, IOException {
+    ReservationDeleteRequestProto requestProto =
+        ((ReservationDeleteRequestPBImpl) request).getProto();
+    try {
+      return new ReservationDeleteResponsePBImpl(proxy.deleteReservation(null,
+          requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
+
 }

+ 61 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java

@@ -45,6 +45,9 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenResponsePBImpl;
@@ -78,6 +81,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQ
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQueuesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -99,6 +108,12 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProt
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProto;
@@ -383,4 +398,50 @@ public class ApplicationClientProtocolPBServiceImpl implements ApplicationClient
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public ReservationSubmissionResponseProto submitReservation(RpcController controller,
+      ReservationSubmissionRequestProto requestProto) throws ServiceException {
+    ReservationSubmissionRequestPBImpl request =
+        new ReservationSubmissionRequestPBImpl(requestProto);
+    try {
+      ReservationSubmissionResponse response = real.submitReservation(request);
+      return ((ReservationSubmissionResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ReservationUpdateResponseProto updateReservation(RpcController controller,
+      ReservationUpdateRequestProto requestProto) throws ServiceException {
+    ReservationUpdateRequestPBImpl request =
+        new ReservationUpdateRequestPBImpl(requestProto);
+    try {
+      ReservationUpdateResponse response = real.updateReservation(request);
+      return ((ReservationUpdateResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ReservationDeleteResponseProto deleteReservation(RpcController controller,
+      ReservationDeleteRequestProto requestProto) throws ServiceException {
+    ReservationDeleteRequestPBImpl request =
+        new ReservationDeleteRequestPBImpl(requestProto);
+    try {
+      ReservationDeleteResponse response = real.deleteReservation(request);
+      return ((ReservationDeleteResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
 }

+ 127 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteRequestPBImpl.java

@@ -0,0 +1,127 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteRequestProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationDeleteRequestPBImpl extends ReservationDeleteRequest {
+
+  ReservationDeleteRequestProto proto = ReservationDeleteRequestProto
+      .getDefaultInstance();
+  ReservationDeleteRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ReservationId reservationId;
+
+  public ReservationDeleteRequestPBImpl() {
+    builder = ReservationDeleteRequestProto.newBuilder();
+  }
+
+  public ReservationDeleteRequestPBImpl(ReservationDeleteRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationDeleteRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationId != null) {
+      builder.setReservationId(convertToProtoFormat(this.reservationId));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationDeleteRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public ReservationId getReservationId() {
+    ReservationDeleteRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationId != null) {
+      return reservationId;
+    }
+    if (!p.hasReservationId()) {
+      return null;
+    }
+    reservationId = convertFromProtoFormat(p.getReservationId());
+    return reservationId;
+  }
+
+  @Override
+  public void setReservationId(ReservationId reservationId) {
+    maybeInitBuilder();
+    if (reservationId == null) {
+      builder.clearReservationId();
+      return;
+    }
+    this.reservationId = reservationId;
+  }
+
+  private ReservationIdPBImpl convertFromProtoFormat(ReservationIdProto p) {
+    return new ReservationIdPBImpl(p);
+  }
+
+  private ReservationIdProto convertToProtoFormat(ReservationId t) {
+    return ((ReservationIdPBImpl) t).getProto();
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

+ 68 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationDeleteResponsePBImpl.java

@@ -0,0 +1,68 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteResponseProto;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationDeleteResponsePBImpl extends ReservationDeleteResponse {
+
+  ReservationDeleteResponseProto proto = ReservationDeleteResponseProto
+      .getDefaultInstance();
+  ReservationDeleteResponseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  public ReservationDeleteResponsePBImpl() {
+    builder = ReservationDeleteResponseProto.newBuilder();
+  }
+
+  public ReservationDeleteResponsePBImpl(ReservationDeleteResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationDeleteResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

+ 152 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionRequestPBImpl.java

@@ -0,0 +1,152 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationDefinitionPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationDefinitionProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionRequestProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationSubmissionRequestPBImpl extends
+    ReservationSubmissionRequest {
+
+  ReservationSubmissionRequestProto proto = ReservationSubmissionRequestProto
+      .getDefaultInstance();
+  ReservationSubmissionRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ReservationDefinition reservationDefinition;
+
+  public ReservationSubmissionRequestPBImpl() {
+    builder = ReservationSubmissionRequestProto.newBuilder();
+  }
+
+  public ReservationSubmissionRequestPBImpl(
+      ReservationSubmissionRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationSubmissionRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationDefinition != null) {
+      builder
+          .setReservationDefinition(convertToProtoFormat(reservationDefinition));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationSubmissionRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public ReservationDefinition getReservationDefinition() {
+    ReservationSubmissionRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationDefinition != null) {
+      return reservationDefinition;
+    }
+    if (!p.hasReservationDefinition()) {
+      return null;
+    }
+    reservationDefinition =
+        convertFromProtoFormat(p.getReservationDefinition());
+    return reservationDefinition;
+  }
+
+  @Override
+  public void setReservationDefinition(
+      ReservationDefinition reservationDefinition) {
+    maybeInitBuilder();
+    if (reservationDefinition == null) {
+      builder.clearReservationDefinition();
+    }
+    this.reservationDefinition = reservationDefinition;
+  }
+
+  @Override
+  public String getQueue() {
+    ReservationSubmissionRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasQueue()) {
+      return null;
+    }
+    return (p.getQueue());
+  }
+
+  @Override
+  public void setQueue(String planName) {
+    maybeInitBuilder();
+    if (planName == null) {
+      builder.clearQueue();
+      return;
+    }
+    builder.setQueue(planName);
+  }
+
+  private ReservationDefinitionProto convertToProtoFormat(
+      ReservationDefinition r) {
+    return ((ReservationDefinitionPBImpl) r).getProto();
+  }
+
+  private ReservationDefinitionPBImpl convertFromProtoFormat(
+      ReservationDefinitionProto r) {
+    return new ReservationDefinitionPBImpl(r);
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

+ 129 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionResponsePBImpl.java

@@ -0,0 +1,129 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionResponseProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationSubmissionResponsePBImpl extends
+    ReservationSubmissionResponse {
+
+  ReservationSubmissionResponseProto proto = ReservationSubmissionResponseProto
+      .getDefaultInstance();
+  ReservationSubmissionResponseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ReservationId reservationId;
+
+  public ReservationSubmissionResponsePBImpl() {
+    builder = ReservationSubmissionResponseProto.newBuilder();
+  }
+
+  public ReservationSubmissionResponsePBImpl(
+      ReservationSubmissionResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationSubmissionResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationId != null) {
+      builder.setReservationId(convertToProtoFormat(this.reservationId));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationSubmissionResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public ReservationId getReservationId() {
+    ReservationSubmissionResponseProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationId != null) {
+      return reservationId;
+    }
+    if (!p.hasReservationId()) {
+      return null;
+    }
+    reservationId = convertFromProtoFormat(p.getReservationId());
+    return reservationId;
+  }
+
+  @Override
+  public void setReservationId(ReservationId reservationId) {
+    maybeInitBuilder();
+    if (reservationId == null) {
+      builder.clearReservationId();
+      return;
+    }
+    this.reservationId = reservationId;
+  }
+
+  private ReservationIdPBImpl convertFromProtoFormat(ReservationIdProto p) {
+    return new ReservationIdPBImpl(p);
+  }
+
+  private ReservationIdProto convertToProtoFormat(ReservationId t) {
+    return ((ReservationIdPBImpl) t).getProto();
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

+ 169 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateRequestPBImpl.java

@@ -0,0 +1,169 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationDefinitionPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationDefinitionProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationUpdateRequestPBImpl extends ReservationUpdateRequest {
+
+  ReservationUpdateRequestProto proto = ReservationUpdateRequestProto
+      .getDefaultInstance();
+  ReservationUpdateRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ReservationDefinition reservationDefinition;
+  private ReservationId reservationId;
+
+  public ReservationUpdateRequestPBImpl() {
+    builder = ReservationUpdateRequestProto.newBuilder();
+  }
+
+  public ReservationUpdateRequestPBImpl(ReservationUpdateRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationUpdateRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationId != null) {
+      builder.setReservationId(convertToProtoFormat(this.reservationId));
+    }
+    if (this.reservationDefinition != null) {
+      builder
+          .setReservationDefinition(convertToProtoFormat(reservationDefinition));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationUpdateRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public ReservationDefinition getReservationDefinition() {
+    ReservationUpdateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationDefinition != null) {
+      return reservationDefinition;
+    }
+    if (!p.hasReservationDefinition()) {
+      return null;
+    }
+    reservationDefinition =
+        convertFromProtoFormat(p.getReservationDefinition());
+    return reservationDefinition;
+  }
+
+  @Override
+  public void setReservationDefinition(
+      ReservationDefinition reservationDefinition) {
+    maybeInitBuilder();
+    if (reservationDefinition == null) {
+      builder.clearReservationDefinition();
+    }
+    this.reservationDefinition = reservationDefinition;
+  }
+
+  @Override
+  public ReservationId getReservationId() {
+    ReservationUpdateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationId != null) {
+      return reservationId;
+    }
+    if (!p.hasReservationId()) {
+      return null;
+    }
+    reservationId = convertFromProtoFormat(p.getReservationId());
+    return reservationId;
+  }
+
+  @Override
+  public void setReservationId(ReservationId reservationId) {
+    maybeInitBuilder();
+    if (reservationId == null) {
+      builder.clearReservationId();
+      return;
+    }
+    this.reservationId = reservationId;
+  }
+
+  private ReservationIdPBImpl convertFromProtoFormat(ReservationIdProto p) {
+    return new ReservationIdPBImpl(p);
+  }
+
+  private ReservationIdProto convertToProtoFormat(ReservationId t) {
+    return ((ReservationIdPBImpl) t).getProto();
+  }
+
+  private ReservationDefinitionProto convertToProtoFormat(
+      ReservationDefinition r) {
+    return ((ReservationDefinitionPBImpl) r).getProto();
+  }
+
+  private ReservationDefinitionPBImpl convertFromProtoFormat(
+      ReservationDefinitionProto r) {
+    return new ReservationDefinitionPBImpl(r);
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

+ 68 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationUpdateResponsePBImpl.java

@@ -0,0 +1,68 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateResponseProto;
+
+import com.google.protobuf.TextFormat;
+
+public class ReservationUpdateResponsePBImpl extends ReservationUpdateResponse {
+
+  ReservationUpdateResponseProto proto = ReservationUpdateResponseProto
+      .getDefaultInstance();
+  ReservationUpdateResponseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  public ReservationUpdateResponsePBImpl() {
+    builder = ReservationUpdateResponseProto.newBuilder();
+  }
+
+  public ReservationUpdateResponsePBImpl(ReservationUpdateResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationUpdateResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

+ 32 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java

@@ -57,6 +57,7 @@ extends ApplicationSubmissionContext {
   private Resource resource = null;
   private Set<String> applicationTags = null;
   private LogAggregationContext logAggregationContext = null;
+  private ReservationId reservationId = null;
 
   public ApplicationSubmissionContextPBImpl() {
     builder = ApplicationSubmissionContextProto.newBuilder();
@@ -114,10 +115,6 @@ extends ApplicationSubmissionContext {
       builder.clearApplicationTags();
       builder.addAllApplicationTags(this.applicationTags);
     }
-    if (this.logAggregationContext != null) {
-      builder.setLogAggregationContext(
-          convertToProtoFormat(this.logAggregationContext));
-    }
   }
 
   private void mergeLocalToProto() {
@@ -365,6 +362,29 @@ extends ApplicationSubmissionContext {
     this.resource = resource;
   }
 
+  @Override
+  public ReservationId getReservationID() {
+    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationId != null) {
+      return reservationId;
+    }
+    if (!p.hasReservationId()) {
+      return null;
+    }
+    reservationId = convertFromProtoFormat(p.getReservationId());
+    return reservationId;
+  }
+
+  @Override
+  public void setReservationID(ReservationId reservationID) {
+    maybeInitBuilder();
+    if (reservationID == null) {
+      builder.clearReservationId();
+      return;
+    }
+    this.reservationId = reservationID;
+  }
+
   @Override
   public void
       setKeepContainersAcrossApplicationAttempts(boolean keepContainers) {
@@ -455,4 +475,12 @@ extends ApplicationSubmissionContext {
       builder.clearLogAggregationContext();
     this.logAggregationContext = logAggregationContext;
   }
+
+  private ReservationIdPBImpl convertFromProtoFormat(ReservationIdProto p) {
+    return new ReservationIdPBImpl(p);
+  }
+
+  private ReservationIdProto convertToProtoFormat(ReservationId t) {
+    return ((ReservationIdPBImpl) t).getProto();
+  }
 }  

+ 17 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueState;
+import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.proto.YarnProtos.AMCommandProto;
@@ -47,10 +48,12 @@ import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
 
 import com.google.protobuf.ByteString;
+
 import org.apache.hadoop.yarn.proto.YarnServiceProtos;
 
 @Private
@@ -237,4 +240,18 @@ public class ProtoUtils {
     return ApplicationAccessType.valueOf(e.name().replace(
         APP_ACCESS_TYPE_PREFIX, ""));
   }
+  
+  /*
+   * Reservation Request interpreter type
+   */
+  public static ReservationRequestInterpreterProto convertToProtoFormat(
+      ReservationRequestInterpreter e) {
+    return ReservationRequestInterpreterProto.valueOf(e.name());
+  }
+
+  public static ReservationRequestInterpreter convertFromProtoFormat(
+      ReservationRequestInterpreterProto e) {
+    return ReservationRequestInterpreter.valueOf(e.name());
+  }
+
 }

+ 169 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java

@@ -0,0 +1,169 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationDefinitionProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationDefinitionProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestsProto;
+
+public class ReservationDefinitionPBImpl extends ReservationDefinition {
+
+  ReservationDefinitionProto proto = ReservationDefinitionProto
+      .getDefaultInstance();
+  ReservationDefinitionProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ReservationRequests reservationReqs;
+
+  public ReservationDefinitionPBImpl() {
+    builder = ReservationDefinitionProto.newBuilder();
+  }
+
+  public ReservationDefinitionPBImpl(ReservationDefinitionProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationDefinitionProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationReqs != null) {
+      builder
+          .setReservationRequests(convertToProtoFormat(this.reservationReqs));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationDefinitionProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public long getArrival() {
+    ReservationDefinitionProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasArrival()) {
+      return 0;
+    }
+    return (p.getArrival());
+  }
+
+  @Override
+  public void setArrival(long earliestStartTime) {
+    maybeInitBuilder();
+    if (earliestStartTime <= 0) {
+      builder.clearArrival();
+      return;
+    }
+    builder.setArrival(earliestStartTime);
+  }
+
+  @Override
+  public long getDeadline() {
+    ReservationDefinitionProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasDeadline()) {
+      return 0;
+    }
+    return (p.getDeadline());
+  }
+
+  @Override
+  public void setDeadline(long latestEndTime) {
+    maybeInitBuilder();
+    if (latestEndTime <= 0) {
+      builder.clearDeadline();
+      return;
+    }
+    builder.setDeadline(latestEndTime);
+  }
+
+  @Override
+  public ReservationRequests getReservationRequests() {
+    ReservationDefinitionProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationReqs != null) {
+      return reservationReqs;
+    }
+    if (!p.hasReservationRequests()) {
+      return null;
+    }
+    reservationReqs = convertFromProtoFormat(p.getReservationRequests());
+    return reservationReqs;
+  }
+
+  @Override
+  public void setReservationRequests(ReservationRequests reservationRequests) {
+    if (reservationRequests == null) {
+      builder.clearReservationRequests();
+      return;
+    }
+    this.reservationReqs = reservationRequests;
+  }
+
+  @Override
+  public String getReservationName() {
+    ReservationDefinitionProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasReservationName()) {
+      return null;
+    }
+    return (p.getReservationName());
+  }
+
+  @Override
+  public void setReservationName(String name) {
+    maybeInitBuilder();
+    if (name == null) {
+      builder.clearReservationName();
+      return;
+    }
+    builder.setReservationName(name);
+  }
+
+  private ReservationRequestsPBImpl convertFromProtoFormat(
+      ReservationRequestsProto p) {
+    return new ReservationRequestsPBImpl(p);
+  }
+
+  private ReservationRequestsProto convertToProtoFormat(ReservationRequests t) {
+    return ((ReservationRequestsPBImpl) t).getProto();
+  }
+
+  @Override
+  public String toString() {
+    return "{Arrival: " + getArrival() + ", Deadline: " + getDeadline()
+        + ", Reservation Name: " + getReservationName() + ", Resources: "
+        + getReservationRequests() + "}";
+  }
+
+}

+ 75 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationIdPBImpl.java

@@ -0,0 +1,75 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
+
+import com.google.common.base.Preconditions;
+
+@Private
+@Unstable
+public class ReservationIdPBImpl extends ReservationId {
+  ReservationIdProto proto = null;
+  ReservationIdProto.Builder builder = null;
+
+  public ReservationIdPBImpl() {
+    builder = ReservationIdProto.newBuilder();
+  }
+
+  public ReservationIdPBImpl(ReservationIdProto proto) {
+    this.proto = proto;
+  }
+
+  public ReservationIdProto getProto() {
+    return proto;
+  }
+
+  @Override
+  public long getId() {
+    Preconditions.checkNotNull(proto);
+    return proto.getId();
+  }
+
+  @Override
+  protected void setId(long id) {
+    Preconditions.checkNotNull(builder);
+    builder.setId(id);
+  }
+
+  @Override
+  public long getClusterTimestamp() {
+    Preconditions.checkNotNull(proto);
+    return proto.getClusterTimestamp();
+  }
+
+  @Override
+  protected void setClusterTimestamp(long clusterTimestamp) {
+    Preconditions.checkNotNull(builder);
+    builder.setClusterTimestamp((clusterTimestamp));
+  }
+
+  @Override
+  protected void build() {
+    proto = builder.build();
+    builder = null;
+  }
+}

+ 152 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestPBImpl.java

@@ -0,0 +1,152 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+
+@Private
+@Unstable
+public class ReservationRequestPBImpl extends ReservationRequest {
+  ReservationRequestProto proto = ReservationRequestProto.getDefaultInstance();
+  ReservationRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private Resource capability = null;
+
+  public ReservationRequestPBImpl() {
+    builder = ReservationRequestProto.newBuilder();
+  }
+
+  public ReservationRequestPBImpl(ReservationRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.capability != null) {
+      builder.setCapability(convertToProtoFormat(this.capability));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public Resource getCapability() {
+    ReservationRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.capability != null) {
+      return this.capability;
+    }
+    if (!p.hasCapability()) {
+      return null;
+    }
+    this.capability = convertFromProtoFormat(p.getCapability());
+    return this.capability;
+  }
+
+  @Override
+  public void setCapability(Resource capability) {
+    maybeInitBuilder();
+    if (capability == null)
+      builder.clearCapability();
+    this.capability = capability;
+  }
+
+  @Override
+  public int getNumContainers() {
+    ReservationRequestProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getNumContainers());
+  }
+
+  @Override
+  public void setNumContainers(int numContainers) {
+    maybeInitBuilder();
+    builder.setNumContainers((numContainers));
+  }
+
+  @Override
+  public int getConcurrency() {
+    ReservationRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasConcurrency()) {
+      return 1;
+    }
+    return (p.getConcurrency());
+  }
+
+  @Override
+  public void setConcurrency(int numContainers) {
+    maybeInitBuilder();
+    builder.setConcurrency(numContainers);
+  }
+
+  @Override
+  public long getDuration() {
+    ReservationRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasDuration()) {
+      return 0;
+    }
+    return (p.getDuration());
+  }
+
+  @Override
+  public void setDuration(long duration) {
+    maybeInitBuilder();
+    builder.setDuration(duration);
+  }
+
+  private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
+    return new ResourcePBImpl(p);
+  }
+
+  private ResourceProto convertToProtoFormat(Resource t) {
+    return ((ResourcePBImpl) t).getProto();
+  }
+
+  @Override
+  public String toString() {
+    return "{Capability: " + getCapability() + ", # Containers: "
+        + getNumContainers() + ", Concurrency: " + getConcurrency()
+        + ", Lease Duration: " + getDuration() + "}";
+  }
+}

+ 189 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationRequestsPBImpl.java

@@ -0,0 +1,189 @@
+/**
+ * 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.api.records.impl.pb;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestsProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestsProtoOrBuilder;
+
+public class ReservationRequestsPBImpl extends ReservationRequests {
+
+  ReservationRequestsProto proto = ReservationRequestsProto
+      .getDefaultInstance();
+  ReservationRequestsProto.Builder builder = null;
+  boolean viaProto = false;
+
+  public List<ReservationRequest> reservationRequests;
+
+  public ReservationRequestsPBImpl() {
+    builder = ReservationRequestsProto.newBuilder();
+  }
+
+  public ReservationRequestsPBImpl(ReservationRequestsProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationRequestsProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationRequests != null) {
+      addReservationResourcesToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationRequestsProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public List<ReservationRequest> getReservationResources() {
+    initReservationRequestsList();
+    return reservationRequests;
+  }
+
+  @Override
+  public void setReservationResources(List<ReservationRequest> resources) {
+    if (resources == null) {
+      builder.clearReservationResources();
+      return;
+    }
+    this.reservationRequests = resources;
+  }
+
+  @Override
+  public ReservationRequestInterpreter getInterpreter() {
+    ReservationRequestsProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasInterpreter()) {
+      return null;
+    }
+    return (convertFromProtoFormat(p.getInterpreter()));
+  }
+
+  @Override
+  public void setInterpreter(ReservationRequestInterpreter interpreter) {
+    maybeInitBuilder();
+    if (interpreter == null) {
+      builder.clearInterpreter();
+      return;
+    }
+    builder.setInterpreter(convertToProtoFormat(interpreter));
+  }
+
+  private void initReservationRequestsList() {
+    if (this.reservationRequests != null) {
+      return;
+    }
+    ReservationRequestsProtoOrBuilder p = viaProto ? proto : builder;
+    List<ReservationRequestProto> resourceProtos =
+        p.getReservationResourcesList();
+    reservationRequests = new ArrayList<ReservationRequest>();
+
+    for (ReservationRequestProto r : resourceProtos) {
+      reservationRequests.add(convertFromProtoFormat(r));
+    }
+  }
+
+  private void addReservationResourcesToProto() {
+    maybeInitBuilder();
+    builder.clearReservationResources();
+    if (reservationRequests == null)
+      return;
+    Iterable<ReservationRequestProto> iterable =
+        new Iterable<ReservationRequestProto>() {
+          @Override
+          public Iterator<ReservationRequestProto> iterator() {
+            return new Iterator<ReservationRequestProto>() {
+
+              Iterator<ReservationRequest> iter = reservationRequests
+                  .iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public ReservationRequestProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+
+            };
+
+          }
+
+        };
+    builder.addAllReservationResources(iterable);
+  }
+
+  private ReservationRequestProto convertToProtoFormat(ReservationRequest r) {
+    return ((ReservationRequestPBImpl) r).getProto();
+  }
+
+  private ReservationRequestPBImpl convertFromProtoFormat(
+      ReservationRequestProto r) {
+    return new ReservationRequestPBImpl(r);
+  }
+
+  private ReservationRequestInterpreterProto convertToProtoFormat(
+      ReservationRequestInterpreter r) {
+    return ProtoUtils.convertToProtoFormat(r);
+  }
+
+  private ReservationRequestInterpreter convertFromProtoFormat(
+      ReservationRequestInterpreterProto r) {
+    return ProtoUtils.convertFromProtoFormat(r);
+  }
+
+  @Override
+  public String toString() {
+    return "{Reservation Resources: " + getReservationResources()
+        + ", Reservation Type: " + getInterpreter() + "}";
+  }
+
+}

+ 39 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/UTCClock.java

@@ -0,0 +1,39 @@
+/**
+ * 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.util;
+
+import java.util.Calendar;
+import java.util.TimeZone;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Implementation of {@link Clock} that gives the current UTC time in
+ * milliseconds.
+ */
+@Public
+@Evolving
+public class UTCClock implements Clock {
+
+  private final TimeZone utcZone = TimeZone.getTimeZone("UTC");
+
+  public long getTime() {
+    return Calendar.getInstance(utcZone).getTimeInMillis();
+  }
+}