diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java index 21c0d0f550..af184807e3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java @@ -44,6 +44,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse; import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest; import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse; import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest; @@ -435,6 +436,12 @@ public void moveApplicationAcrossQueues(ApplicationId appId, String queue) client.moveApplicationAcrossQueues(appId, queue); } + @Override + public GetNewReservationResponse createReservation() throws YarnException, + IOException { + return client.createReservation(); + } + @Override public ReservationSubmissionResponse submitReservation( ReservationSubmissionRequest request) throws YarnException, IOException { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java index 64f967dbf9..bed7e75a78 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java @@ -96,6 +96,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; @@ -423,6 +425,12 @@ public GetContainersResponse getContainers(GetContainersRequest request) return null; } + @Override + public GetNewReservationResponse getNewReservation( + GetNewReservationRequest request) throws YarnException, IOException { + return null; + } + @Override public ReservationSubmissionResponse submitReservation( ReservationSubmissionRequest request) throws YarnException, IOException { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java index bca062e7fa..82b27085cc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java @@ -37,6 +37,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; @@ -301,6 +303,28 @@ public GetQueueUserAclsInfoResponse getQueueUserAcls( public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues( MoveApplicationAcrossQueuesRequest request) throws YarnException, IOException; + /** + *
The interface used by clients to obtain a new {@link ReservationId} for + * submitting new reservations.
+ * + *The ResourceManager
responds with a new, unique,
+ * {@link ReservationId} which is used by the client to submit
+ * a new reservation.
ReservationId
+ * @return response containing the new ReservationId
to be used
+ * to submit a new reservation
+ * @throws YarnException if the reservation system is not enabled.
+ * @throws IOException on IO failures.
+ * @see #submitReservation(ReservationSubmissionRequest)
+ */
+ @Public
+ @Unstable
+ @Idempotent
+ GetNewReservationResponse getNewReservation(
+ GetNewReservationRequest request)
+ throws YarnException, IOException;
+
/**
* * The interface used by clients to submit a new reservation to the @@ -349,6 +373,7 @@ public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues( */ @Public @Unstable + @Idempotent public ReservationSubmissionResponse submitReservation( ReservationSubmissionRequest request) throws YarnException, IOException; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewReservationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewReservationRequest.java new file mode 100644 index 0000000000..210351dd8d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewReservationRequest.java @@ -0,0 +1,41 @@ +/** + * 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.util.Records; + +/** + *
The request sent by clients to get a new {@code ReservationId} for + * submitting an reservation.
+ * + * {@code ApplicationClientProtocol#getNewReservation(GetNewReservationRequest)} + */ +@Public +@Unstable +public abstract class GetNewReservationRequest { + @Public + @Unstable + public static GetNewReservationRequest newInstance() { + GetNewReservationRequest request = + Records.newRecord(GetNewReservationRequest.class); + return request; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewReservationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewReservationResponse.java new file mode 100644 index 0000000000..d7d80bff4b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewReservationResponse.java @@ -0,0 +1,71 @@ +/** + * 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.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ReservationId; +import org.apache.hadoop.yarn.util.Records; + +/** + *The response sent by the ResourceManager
to the client for
+ * a request to get a new {@link ReservationId} for submitting reservations.
Clients can submit an reservation with the returned + * {@link ReservationId}.
+ * + * {@code ApplicationClientProtocol#getNewReservation(GetNewReservationRequest)} + */ +@Public +@Unstable +public abstract class GetNewReservationResponse { + + @Private + @Unstable + public static GetNewReservationResponse newInstance( + ReservationId reservationId) { + GetNewReservationResponse response = + Records.newRecord(GetNewReservationResponse.class); + response.setReservationId(reservationId); + return response; + } + + /** + * Get a new {@link ReservationId} to be used to submit a reservation. + * + * @return a {@link ReservationId} representing the unique id to identify + * a reservation with which it was submitted. + */ + @Public + @Unstable + public abstract ReservationId getReservationId(); + + /** + * Set a new {@link ReservationId} to be used to submit a reservation. + * + * @param reservationId a {@link ReservationId} representing the unique id to + * identify a reservation with which it was submitted. + */ + @Private + @Unstable + public abstract void setReservationId(ReservationId reservationId); + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java index 6d5ca16744..3872f41a26 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionRequest.java @@ -22,6 +22,7 @@ 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.api.records.ReservationId; import org.apache.hadoop.yarn.util.Records; /** @@ -38,11 +39,13 @@ public abstract class ReservationSubmissionRequest { @Public @Unstable public static ReservationSubmissionRequest newInstance( - ReservationDefinition reservationDefinition, String queueName) { + ReservationDefinition reservationDefinition, String queueName, + ReservationId reservationId) { ReservationSubmissionRequest request = Records.newRecord(ReservationSubmissionRequest.class); request.setReservationDefinition(reservationDefinition); request.setQueue(queueName); + request.setReservationId(reservationId); return request; } @@ -94,4 +97,24 @@ public abstract void setReservationDefinition( @Unstable public abstract void setQueue(String queueName); + /** + * Get the reservation id that corresponds to the reservation submission. + * + * @return reservation id that will be used to identify the reservation + * submission. + */ + @Public + @Unstable + public abstract ReservationId getReservationId(); + + /** + * Set the reservation id that corresponds to the reservation submission. + * + * @param reservationId reservation id that will be used to identify the + * reservation submission. + */ + @Public + @Unstable + public abstract void setReservationId(ReservationId reservationId); + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java index 32fe5e0b2c..87592d0ed2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ReservationSubmissionResponse.java @@ -21,18 +21,17 @@ 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; /** - * {@link ReservationSubmissionResponse} contains the answer of the admission - * control system in the {@code 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 - * + *The response sent by the ResourceManager
to a client on
+ * reservation submission.
Currently, this is empty.
+ * + * {@code ApplicationClientProtocol#submitReservation( + * ReservationSubmissionRequest)} + * */ @Public @Unstable @@ -40,37 +39,10 @@ public abstract class ReservationSubmissionResponse { @Private @Unstable - public static ReservationSubmissionResponse newInstance( - ReservationId reservationId) { + public static ReservationSubmissionResponse newInstance() { 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); - } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto index 763c839dd8..7046b2490d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto @@ -50,6 +50,7 @@ service ApplicationClientProtocolService { rpc getApplicationAttempts (GetApplicationAttemptsRequestProto) returns (GetApplicationAttemptsResponseProto); rpc getContainerReport (GetContainerReportRequestProto) returns (GetContainerReportResponseProto); rpc getContainers (GetContainersRequestProto) returns (GetContainersResponseProto); + rpc getNewReservation (GetNewReservationRequestProto) returns (GetNewReservationResponseProto); rpc submitReservation (ReservationSubmissionRequestProto) returns (ReservationSubmissionResponseProto); rpc updateReservation (ReservationUpdateRequestProto) returns (ReservationUpdateResponseProto); rpc deleteReservation (ReservationDeleteRequestProto) returns (ReservationDeleteResponseProto); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto index bdf022fb7e..7070e38b1c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto @@ -367,13 +367,20 @@ message ReleaseSharedCacheResourceResponseProto { // reservation_protocol ////////////////////////////////////////////////////// +message GetNewReservationRequestProto { +} + +message GetNewReservationResponseProto { + optional ReservationIdProto reservation_id = 1; +} + message ReservationSubmissionRequestProto { optional string queue = 1; optional ReservationDefinitionProto reservation_definition = 2; + optional ReservationIdProto reservation_id = 3; } message ReservationSubmissionResponseProto { - optional ReservationIdProto reservation_id = 1; } message ReservationUpdateRequestProto { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java index ff231a8281..7dfe1d9b25 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java @@ -32,6 +32,7 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse; import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest; import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse; import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest; @@ -533,6 +534,20 @@ public abstract List+ * Obtain a {@link GetNewReservationResponse} for a new reservation, + * which contains the {@link ReservationId} object. + *
+ * + * @return The {@link GetNewReservationResponse} containing a new + * {@link ReservationId} object. + * @throws YarnException if reservation cannot be created. + * @throws IOException if reservation cannot be created. + */ + public abstract GetNewReservationResponse createReservation() + throws YarnException, IOException; + /** *
* The interface used by clients to submit a new reservation to the
@@ -666,7 +681,7 @@ public abstract ReservationDeleteResponse deleteReservation(
* @return response that contains information about reservations that are
* being searched for.
* @throws YarnException if the request is invalid
- * @throws IOException
+ * @throws IOException if the request failed otherwise
*
*/
@Public
@@ -725,8 +740,10 @@ public abstract Map
The implementation of the request sent by clients to get a + * new {@code ReservationId} for submitting an reservation.
+ * + * {@code ApplicationClientProtocol#getNewReservation(GetNewReservationRequest)} + */ +@Private +@Unstable +public class GetNewReservationRequestPBImpl extends GetNewReservationRequest { + private GetNewReservationRequestProto proto = + GetNewReservationRequestProto.getDefaultInstance(); + private GetNewReservationRequestProto.Builder builder = null; + private boolean viaProto = false; + + public GetNewReservationRequestPBImpl(GetNewReservationRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetNewReservationRequestPBImpl() { + builder = GetNewReservationRequestProto.newBuilder(); + } + + public GetNewReservationRequestProto 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()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNewReservationResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNewReservationResponsePBImpl.java new file mode 100644 index 0000000000..c09c4e214c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNewReservationResponsePBImpl.java @@ -0,0 +1,144 @@ +/** + * 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 com.google.protobuf.TextFormat; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse; +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.GetNewReservationResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewReservationResponseProtoOrBuilder; + + +/** + *The implementation of the response sent by the
+ * ResourceManager
to the client for a request to get a new
+ * {@link ReservationId} for submitting reservations.
Clients can submit an reservation with the returned + * {@link ReservationId}.
+ * + * {@code ApplicationClientProtocol#getNewReservation(GetNewReservationRequest)} + */ +@Private +@Unstable +public class GetNewReservationResponsePBImpl extends GetNewReservationResponse { + private GetNewReservationResponseProto proto = + GetNewReservationResponseProto.getDefaultInstance(); + private GetNewReservationResponseProto.Builder builder = null; + private boolean viaProto = false; + + private ReservationId reservationId = null; + + public GetNewReservationResponsePBImpl() { + builder = GetNewReservationResponseProto.newBuilder(); + } + + public GetNewReservationResponsePBImpl(GetNewReservationResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetNewReservationResponseProto getProto() { + mergeLocalToProto(); + 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()); + } + + private void mergeLocalToBuilder() { + if (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 = GetNewReservationResponseProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public ReservationId getReservationId() { + if (this.reservationId != null) { + return this.reservationId; + } + + GetNewReservationResponseProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasReservationId()) { + return null; + } + + this.reservationId = convertFromProtoFormat(p.getReservationId()); + return this.reservationId; + } + + @Override + public void setReservationId(ReservationId reservationId) { + maybeInitBuilder(); + if (reservationId == null) { + builder.clearReservationId(); + } + this.reservationId = reservationId; + } + + private ReservationIdPBImpl convertFromProtoFormat(ReservationIdProto p) { + return new ReservationIdPBImpl(p); + } + + private ReservationIdProto convertToProtoFormat(ReservationId t) { + return ((ReservationIdPBImpl)t).getProto(); + } + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionRequestPBImpl.java index 9f10f4c75a..016d4de4f0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionRequestPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionRequestPBImpl.java @@ -20,8 +20,11 @@ import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest; 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.ReservationSubmissionRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionRequestProtoOrBuilder; @@ -119,6 +122,25 @@ public void setQueue(String planName) { builder.setQueue(planName); } + @Override + public ReservationId getReservationId() { + ReservationSubmissionRequestProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasReservationId()) { + return null; + } + return (convertFromProtoFormat(p.getReservationId())); + } + + @Override + public void setReservationId(ReservationId reservationId) { + maybeInitBuilder(); + if (reservationId == null) { + builder.clearReservationId(); + return; + } + builder.setReservationId(convertToProtoFormat(reservationId)); + } + private ReservationDefinitionProto convertToProtoFormat( ReservationDefinition r) { return ((ReservationDefinitionPBImpl) r).getProto(); @@ -129,6 +151,14 @@ private ReservationDefinitionPBImpl convertFromProtoFormat( return new ReservationDefinitionPBImpl(r); } + private ReservationIdProto convertToProtoFormat(ReservationId r) { + return ((ReservationIdPBImpl) r).getProto(); + } + + private ReservationIdPBImpl convertFromProtoFormat(ReservationIdProto r) { + return new ReservationIdPBImpl(r); + } + @Override public int hashCode() { return getProto().hashCode(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionResponsePBImpl.java index 82151f4046..81b3a84dd4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionResponsePBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ReservationSubmissionResponsePBImpl.java @@ -19,11 +19,7 @@ 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; @@ -35,8 +31,6 @@ public class ReservationSubmissionResponsePBImpl extends ReservationSubmissionResponseProto.Builder builder = null; boolean viaProto = false; - private ReservationId reservationId; - public ReservationSubmissionResponsePBImpl() { builder = ReservationSubmissionResponseProto.newBuilder(); } @@ -48,64 +42,11 @@ public ReservationSubmissionResponsePBImpl( } 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(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java index c512f8dff3..6420cb0eb0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java @@ -61,6 +61,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; @@ -425,6 +427,12 @@ public GetContainersResponse getContainers(GetContainersRequest request) throw new NotImplementedException(); } + @Override + public GetNewReservationResponse getNewReservation( + GetNewReservationRequest request) throws YarnException, IOException { + throw new NotImplementedException(); + } + @Override public ReservationSubmissionResponse submitReservation( ReservationSubmissionRequest request) throws YarnException, @@ -488,4 +496,4 @@ public FailApplicationAttemptResponse failApplicationAttempt( FailApplicationAttemptRequest request) throws YarnException, IOException { throw new NotImplementedException(); } -} \ No newline at end of file +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index d68b7b1e59..aeeea2b53e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -79,6 +79,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; @@ -1217,6 +1219,19 @@ public Server getServer() { return this.server; } + @Override + public GetNewReservationResponse getNewReservation( + GetNewReservationRequest request) throws YarnException, IOException { + checkReservationSytem(AuditConstants.CREATE_NEW_RESERVATION_REQUEST); + GetNewReservationResponse response = + recordFactory.newRecordInstance(GetNewReservationResponse.class); + + ReservationId reservationId = reservationSystem.getNewReservationId(); + response.setReservationId(reservationId); + // Create a new Reservation Id + return response; + } + @Override public ReservationSubmissionResponse submitReservation( ReservationSubmissionRequest request) throws YarnException, IOException { @@ -1224,12 +1239,28 @@ public ReservationSubmissionResponse submitReservation( checkReservationSytem(AuditConstants.SUBMIT_RESERVATION_REQUEST); ReservationSubmissionResponse response = recordFactory.newRecordInstance(ReservationSubmissionResponse.class); - // Create a new Reservation Id - ReservationId reservationId = reservationSystem.getNewReservationId(); + ReservationId reservationId = request.getReservationId(); // Validate the input Plan plan = rValidator.validateReservationSubmissionRequest(reservationSystem, request, reservationId); + + ReservationAllocation allocation = plan.getReservationById(reservationId); + + if (allocation != null) { + boolean isNewDefinition = !allocation.getReservationDefinition().equals( + request.getReservationDefinition()); + if (isNewDefinition) { + String message = "Reservation allocation already exists with the " + + "reservation id " + reservationId.toString() + ", but a different" + + " reservation definition was provided. Please try again with a " + + "new reservation id, or consider updating the reservation instead."; + throw RPCUtil.getRemoteException(message); + } else { + return response; + } + } + // Check ACLs String queueName = request.getQueue(); String user = @@ -1248,7 +1279,6 @@ public ReservationSubmissionResponse submitReservation( refreshScheduler(queueName, request.getReservationDefinition(), reservationId.toString()); // return the reservation id - response.setReservationId(reservationId); } } catch (PlanningException e) { RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_RESERVATION_REQUEST, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java index d08cb9eaec..c5bf00068d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java @@ -69,6 +69,8 @@ public static class AuditConstants { public static final String UNAUTHORIZED_USER = "Unauthorized user"; // For Reservation system + public static final String CREATE_NEW_RESERVATION_REQUEST = "Create " + + "Reservation Request"; public static final String SUBMIT_RESERVATION_REQUEST = "Submit Reservation Request"; public static final String UPDATE_RESERVATION_REQUEST = "Update Reservation Request"; public static final String DELETE_RESERVATION_REQUEST = "Delete Reservation Request"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java index d63e725015..8ebde0ad52 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java @@ -206,10 +206,17 @@ public Plan validateReservationSubmissionRequest( ReservationSystem reservationSystem, ReservationSubmissionRequest request, ReservationId reservationId) throws YarnException { + String message; + if (reservationId == null) { + message = "Reservation id cannot be null. Please try again " + + "specifying a valid reservation id by creating a new reservation id."; + throw RPCUtil.getRemoteException(message); + } // Check if it is a managed queue String queue = request.getQueue(); Plan plan = getPlanFromQueue(reservationSystem, queue, AuditConstants.SUBMIT_RESERVATION_REQUEST); + validateReservationDefinition(reservationId, request.getReservationDefinition(), plan, AuditConstants.SUBMIT_RESERVATION_REQUEST); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java index 50c450bf6c..a4c01df448 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java @@ -78,6 +78,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse; import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest; @@ -149,6 +151,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LocalResourceInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LogAggregationContextInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication; +import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewReservation; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo; @@ -162,7 +165,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationListInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationRequestInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationRequestsInfo; -import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionResponseInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateResponseInfo; @@ -1869,6 +1871,61 @@ private TokenThe response sent by the ResourceManager
to the client for
+ * a request to get a new {@code ReservationId} for submitting reservations
+ * using the REST API.
Clients can submit a reservation with the returned {@code ReservationId}. + *
+ * + * {@code RMWebServices#createNewReservation(HttpServletRequest)} + */ +@XmlRootElement(name="new-reservation") +@XmlAccessorType(XmlAccessType.FIELD) +public class NewReservation { + + @XmlElement(name="reservation-id") + private String reservationId; + + public NewReservation() { + reservationId = ""; + } + + public NewReservation(String resId) { + reservationId = resId; + } + + public String getReservationId() { + return reservationId; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionRequestInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionRequestInfo.java index 701370dc35..5ca799858b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionRequestInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionRequestInfo.java @@ -37,6 +37,9 @@ public class ReservationSubmissionRequestInfo { @XmlElement(name = "reservation-definition") private ReservationDefinitionInfo reservationDefinition; + @XmlElement(name = "reservation-id") + private String reservationId; + public ReservationSubmissionRequestInfo() { } @@ -48,6 +51,14 @@ public void setQueue(String queue) { this.queue = queue; } + public String getReservationId() { + return reservationId; + } + + public void setReservationId(String reservationId) { + this.reservationId = reservationId; + } + public ReservationDefinitionInfo getReservationDefinition() { return reservationDefinition; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionResponseInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionResponseInfo.java deleted file mode 100644 index 943390b350..0000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationSubmissionResponseInfo.java +++ /dev/null @@ -1,54 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao; - -import javax.xml.bind.annotation.XmlAccessType; -import javax.xml.bind.annotation.XmlAccessorType; -import javax.xml.bind.annotation.XmlElement; -import javax.xml.bind.annotation.XmlRootElement; - -import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse; - -/** - * Simple class that represent a response to a reservation submission. - */ -@XmlRootElement(name = "reservation-submission-response") -@XmlAccessorType(XmlAccessType.FIELD) -public class ReservationSubmissionResponseInfo { - - @XmlElement(name = "reservation-id") - private String reservationId; - - public ReservationSubmissionResponseInfo() { - - } - - public ReservationSubmissionResponseInfo( - ReservationSubmissionResponse response) { - this.reservationId = response.getReservationId().toString(); - } - - public String getReservationId() { - return reservationId; - } - - public void setReservationId(String reservationId) { - this.reservationId = reservationId; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java index 4039f50cb8..518da6105a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ReservationACLsTestBase.java @@ -32,6 +32,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse; import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest; import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest; import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse; @@ -242,8 +244,8 @@ public void testApplicationACLs() throws Exception { private void verifySubmitReservationSuccess(String submitter, String queueName) throws Exception { - ReservationId reservationId = - submitReservation(submitter, queueName); + ReservationId reservationId = createReservation(submitter); + submitReservation(submitter, queueName, reservationId); deleteReservation(submitter, reservationId); } @@ -251,7 +253,8 @@ private void verifySubmitReservationSuccess(String submitter, String private void verifySubmitReservationFailure(String submitter, String queueName) throws Exception { try { - submitReservation(submitter, queueName); + ReservationId reservationId = createReservation(submitter); + submitReservation(submitter, queueName, reservationId); Assert.fail("Submit reservation by the enemy should fail!"); } catch (YarnException e) { handleAdministerException(e, submitter, queueName, ReservationACL @@ -261,8 +264,8 @@ private void verifySubmitReservationFailure(String submitter, String private void verifyListReservationSuccess(String lister, String originalSubmitter, String queueName) throws Exception { - ReservationId reservationId = - submitReservation(originalSubmitter, queueName); + ReservationId reservationId = createReservation(originalSubmitter); + submitReservation(originalSubmitter, queueName, reservationId); ReservationListResponse adminResponse = listReservation(lister, queueName); @@ -275,8 +278,8 @@ private void verifyListReservationSuccess(String lister, String private void verifyListReservationFailure(String lister, String originalSubmitter, String queueName) throws Exception { - ReservationId reservationId = - submitReservation(originalSubmitter, queueName); + ReservationId reservationId = createReservation(originalSubmitter); + submitReservation(originalSubmitter, queueName, reservationId); try { listReservation(lister, queueName); @@ -291,8 +294,8 @@ private void verifyListReservationFailure(String lister, private void verifyListReservationByIdSuccess(String lister, String originalSubmitter, String queueName) throws Exception { - ReservationId reservationId = - submitReservation(originalSubmitter, queueName); + ReservationId reservationId = createReservation(originalSubmitter); + submitReservation(originalSubmitter, queueName, reservationId); ReservationListResponse adminResponse = listReservationById(lister, reservationId, queueName); @@ -306,8 +309,8 @@ private void verifyListReservationByIdSuccess(String lister, String private void verifyListReservationByIdFailure(String lister, String originalSubmitter, String queueName) throws Exception { - ReservationId reservationId = - submitReservation(originalSubmitter, queueName); + ReservationId reservationId = createReservation(originalSubmitter); + submitReservation(originalSubmitter, queueName, reservationId); try { listReservationById(lister, reservationId, queueName); Assert.fail("List reservation by the enemy should fail!"); @@ -321,8 +324,8 @@ private void verifyListReservationByIdFailure(String lister, private void verifyDeleteReservationSuccess(String killer, String originalSubmitter, String queueName) throws Exception { - ReservationId reservationId = - submitReservation(originalSubmitter, queueName); + ReservationId reservationId = createReservation(originalSubmitter); + submitReservation(originalSubmitter, queueName, reservationId); deleteReservation(killer, reservationId); } @@ -330,8 +333,8 @@ private void verifyDeleteReservationSuccess(String killer, private void verifyDeleteReservationFailure(String killer, String originalSubmitter, String queueName) throws Exception { - ReservationId reservationId = - submitReservation(originalSubmitter, queueName); + ReservationId reservationId = createReservation(originalSubmitter); + submitReservation(originalSubmitter, queueName, reservationId); try { deleteReservation(killer, reservationId); @@ -346,8 +349,8 @@ private void verifyDeleteReservationFailure(String killer, private void verifyUpdateReservationSuccess(String updater, String originalSubmitter, String queueName) throws Exception { - ReservationId reservationId = - submitReservation(originalSubmitter, queueName); + ReservationId reservationId = createReservation(originalSubmitter); + submitReservation(originalSubmitter, queueName, reservationId); final ReservationUpdateRequest updateRequest = ReservationUpdateRequest.newInstance( @@ -362,8 +365,8 @@ private void verifyUpdateReservationSuccess(String updater, private void verifyUpdateReservationFailure(String updater, String originalSubmitter, String queueName) throws Exception { - ReservationId reservationId = - submitReservation(originalSubmitter, queueName); + ReservationId reservationId = createReservation(originalSubmitter); + submitReservation(originalSubmitter, queueName, reservationId); final ReservationUpdateRequest updateRequest = ReservationUpdateRequest.newInstance( @@ -422,17 +425,27 @@ private void deleteReservation(String deleter, ReservationId id) throws deleteClient.deleteReservation(deleteRequest); } - private ReservationId submitReservation(String submitter, - String queueName) throws Exception { + private ReservationId createReservation(String creator) throws Exception { + + ApplicationClientProtocol creatorClient = getRMClientForUser(creator); + GetNewReservationRequest getNewReservationRequest = + GetNewReservationRequest.newInstance(); + + GetNewReservationResponse response = creatorClient + .getNewReservation(getNewReservationRequest); + return response.getReservationId(); + } + + private void submitReservation(String submitter, + String queueName, ReservationId reservationId) throws Exception { ApplicationClientProtocol submitterClient = getRMClientForUser(submitter); ReservationSubmissionRequest reservationSubmissionRequest = - ReservationSubmissionRequest.newInstance( - makeSimpleReservationDefinition(), queueName); + ReservationSubmissionRequest.newInstance( + makeSimpleReservationDefinition(), queueName, reservationId); ReservationSubmissionResponse response = submitterClient .submitReservation(reservationSubmissionRequest); - return response.getReservationId(); } private void handleAdministerException(Exception e, String user, String diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java index 8a6ddae29b..f5826c1a97 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java @@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; @@ -1091,9 +1092,7 @@ private static YarnScheduler mockYarnScheduler() { return yarnScheduler; } - @Test - public void testReservationAPIs() { - // initialize + private ResourceManager setupResourceManager() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); ReservationSystemTestUtil.setupQueueConfiguration(conf); conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, @@ -1101,41 +1100,95 @@ public void testReservationAPIs() { conf.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true); MockRM rm = new MockRM(conf); rm.start(); - MockNM nm; try { - nm = rm.registerNode("127.0.0.1:1", 102400, 100); + rm.registerNode("127.0.0.1:1", 102400, 100); // allow plan follower to synchronize Thread.sleep(1050); } catch (Exception e) { Assert.fail(e.getMessage()); } + return rm; + } - // Create a client. - ClientRMService clientService = rm.getClientRMService(); - - // create a reservation - Clock clock = new UTCClock(); - long arrival = clock.getTime(); - long duration = 60000; - long deadline = (long) (arrival + 1.05 * duration); - ReservationSubmissionRequest sRequest = - ReservationSystemTestUtil.createSimpleReservationRequest(4, arrival, - deadline, duration); + private ReservationSubmissionRequest submitReservationTestHelper( + ClientRMService clientService, long arrival, long deadline, + long duration) { ReservationSubmissionResponse sResponse = null; + GetNewReservationRequest newReservationRequest = + GetNewReservationRequest.newInstance(); + ReservationId reservationID = null; + try { + reservationID = clientService.getNewReservation(newReservationRequest) + .getReservationId(); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + ReservationSubmissionRequest sRequest = + ReservationSystemTestUtil.createSimpleReservationRequest(reservationID, + 4, arrival, deadline, duration); try { sResponse = clientService.submitReservation(sRequest); } catch (Exception e) { Assert.fail(e.getMessage()); } Assert.assertNotNull(sResponse); - ReservationId reservationID = sResponse.getReservationId(); Assert.assertNotNull(reservationID); - LOG.info("Submit reservation response: " + reservationID); + System.out.println("Submit reservation response: " + reservationID); + return sRequest; + } + + @Test + public void testCreateReservation() { + ResourceManager rm = setupResourceManager(); + ClientRMService clientService = rm.getClientRMService(); + Clock clock = new UTCClock(); + long arrival = clock.getTime(); + long duration = 60000; + long deadline = (long) (arrival + 1.05 * duration); + ReservationSubmissionRequest sRequest = + submitReservationTestHelper(clientService, arrival, deadline, duration); + + // Submit the reservation again with the same request and make sure it + // passes. + try { + clientService.submitReservation(sRequest); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + + // Submit the reservation with the same reservation id but different + // reservation definition, and ensure YarnException is thrown. + arrival = clock.getTime(); + ReservationDefinition rDef = sRequest.getReservationDefinition(); + rDef.setArrival(arrival + duration); + sRequest.setReservationDefinition(rDef); + try { + clientService.submitReservation(sRequest); + Assert.fail("Reservation submission should fail if a duplicate " + + "reservation id is used, but the reservation definition has been " + + "updated."); + } catch (Exception e) { + Assert.assertTrue(e instanceof YarnException); + } + + rm.stop(); + } + + @Test + public void testUpdateReservation() { + ResourceManager rm = setupResourceManager(); + ClientRMService clientService = rm.getClientRMService(); + Clock clock = new UTCClock(); + long arrival = clock.getTime(); + long duration = 60000; + long deadline = (long) (arrival + 1.05 * duration); + ReservationSubmissionRequest sRequest = + submitReservationTestHelper(clientService, arrival, deadline, duration); - // Update the reservation ReservationDefinition rDef = sRequest.getReservationDefinition(); ReservationRequest rr = rDef.getReservationRequests().getReservationResources().get(0); + ReservationId reservationID = sRequest.getReservationId(); rr.setNumContainers(5); arrival = clock.getTime(); duration = 30000; @@ -1151,14 +1204,61 @@ public void testReservationAPIs() { } catch (Exception e) { Assert.fail(e.getMessage()); } - Assert.assertNotNull(sResponse); - LOG.info("Update reservation response: " + uResponse); + Assert.assertNotNull(uResponse); + System.out.println("Update reservation response: " + uResponse); - // List reservations, search by reservation ID - ReservationListRequest request = - ReservationListRequest.newInstance( - ReservationSystemTestUtil.reservationQ, - reservationID.toString(), -1, -1, false); + rm.stop(); + } + + @Test + public void testListReservationsByReservationId() { + ResourceManager rm = setupResourceManager(); + ClientRMService clientService = rm.getClientRMService(); + Clock clock = new UTCClock(); + long arrival = clock.getTime(); + long duration = 60000; + long deadline = (long) (arrival + 1.05 * duration); + ReservationSubmissionRequest sRequest = + submitReservationTestHelper(clientService, arrival, deadline, duration); + + ReservationId reservationID = sRequest.getReservationId(); + ReservationListResponse response = null; + ReservationListRequest request = ReservationListRequest.newInstance( + ReservationSystemTestUtil.reservationQ, reservationID.toString(), -1, + -1, false); + try { + response = clientService.listReservations(request); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + Assert.assertNotNull(response); + Assert.assertEquals(1, response.getReservationAllocationState().size()); + Assert.assertEquals(response.getReservationAllocationState().get(0) + .getReservationId().getId(), reservationID.getId()); + Assert.assertEquals(response.getReservationAllocationState().get(0) + .getResourceAllocationRequests().size(), 0); + + rm.stop(); + } + + @Test + public void testListReservationsByTimeInterval() { + ResourceManager rm = setupResourceManager(); + ClientRMService clientService = rm.getClientRMService(); + Clock clock = new UTCClock(); + long arrival = clock.getTime(); + long duration = 60000; + long deadline = (long) (arrival + 1.05 * duration); + ReservationSubmissionRequest sRequest = + submitReservationTestHelper(clientService, arrival, deadline, duration); + + // List reservations, search by a point in time within the reservation + // range. + arrival = clock.getTime(); + ReservationId reservationID = sRequest.getReservationId(); + ReservationListRequest request = ReservationListRequest.newInstance( + ReservationSystemTestUtil.reservationQ, "", arrival + duration / 2, + arrival + duration / 2, true); ReservationListResponse response = null; try { @@ -1167,16 +1267,12 @@ public void testReservationAPIs() { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - Assert.assertEquals(response.getReservationAllocationState().size(), 1); + Assert.assertEquals(1, response.getReservationAllocationState().size()); Assert.assertEquals(response.getReservationAllocationState().get(0) - .getReservationId().getId(), reservationID.getId()); - Assert.assertEquals(response.getReservationAllocationState().get(0) - .getResourceAllocationRequests().size(), 0); - + .getReservationId().getId(), reservationID.getId()); // List reservations, search by time within reservation interval. request = ReservationListRequest.newInstance( - ReservationSystemTestUtil.reservationQ, "", 1, Long.MAX_VALUE, - true); + ReservationSystemTestUtil.reservationQ, "", 1, Long.MAX_VALUE, true); response = null; try { @@ -1187,73 +1283,52 @@ public void testReservationAPIs() { Assert.assertNotNull(response); Assert.assertEquals(1, response.getReservationAllocationState().size()); Assert.assertEquals(response.getReservationAllocationState().get(0) - .getReservationId().getId(), reservationID.getId()); - - // List reservations, search by invalid end time == -1. - request = ReservationListRequest.newInstance( - ReservationSystemTestUtil.reservationQ, "", 1, -1, - true); - - response = null; - try { - response = clientService.listReservations(request); - } catch (Exception e) { - Assert.fail(e.getMessage()); - } - Assert.assertNotNull(response); - Assert.assertEquals(1, response.getReservationAllocationState().size()); - Assert.assertEquals(response.getReservationAllocationState().get(0) - .getReservationId().getId(), reservationID.getId()); - - // List reservations, search by invalid end time < -1. - request = ReservationListRequest.newInstance( - ReservationSystemTestUtil.reservationQ, "", 1, -10, - true); - - response = null; - try { - response = clientService.listReservations(request); - } catch (Exception e) { - Assert.fail(e.getMessage()); - } - Assert.assertNotNull(response); - Assert.assertEquals(1, response.getReservationAllocationState().size()); - Assert.assertEquals(response.getReservationAllocationState().get(0) - .getReservationId().getId(), reservationID.getId()); - - // List reservations, search by time interval. - request = ReservationListRequest.newInstance( - ReservationSystemTestUtil.reservationQ, "", arrival + - duration/2, arrival + duration/2, true); - - response = null; - try { - response = clientService.listReservations(request); - } catch (Exception e) { - Assert.fail(e.getMessage()); - } - Assert.assertNotNull(response); - Assert.assertEquals(1, response.getReservationAllocationState().size()); - Assert.assertEquals(response.getReservationAllocationState().get(0) - .getReservationId().getId(), reservationID.getId()); - + .getReservationId().getId(), reservationID.getId()); // Verify that the full resource allocations exist. Assert.assertTrue(response.getReservationAllocationState().get(0) - .getResourceAllocationRequests().size() > 0); + .getResourceAllocationRequests().size() > 0); // Verify that the full RDL is returned. - ReservationRequests reservationRequests = response - .getReservationAllocationState().get(0).getReservationDefinition() - .getReservationRequests(); - Assert.assertTrue(reservationRequests.getInterpreter().toString() - .equals("R_ALL")); + ReservationRequests reservationRequests = + response.getReservationAllocationState().get(0) + .getReservationDefinition().getReservationRequests(); + Assert.assertTrue( + reservationRequests.getInterpreter().toString().equals("R_ALL")); Assert.assertTrue(reservationRequests.getReservationResources().get(0) - .getDuration() == duration); + .getDuration() == duration); - // List reservations, search by a very large start time. - request = ReservationListRequest.newInstance( - ReservationSystemTestUtil.reservationQ, "", Long.MAX_VALUE, - -1, false); + rm.stop(); + } + + @Test + public void testListReservationsByInvalidTimeInterval() { + ResourceManager rm = setupResourceManager(); + ClientRMService clientService = rm.getClientRMService(); + Clock clock = new UTCClock(); + long arrival = clock.getTime(); + long duration = 60000; + long deadline = (long) (arrival + 1.05 * duration); + ReservationSubmissionRequest sRequest = + submitReservationTestHelper(clientService, arrival, deadline, duration); + + // List reservations, search by invalid end time == -1. + ReservationListRequest request = ReservationListRequest + .newInstance(ReservationSystemTestUtil.reservationQ, "", 1, -1, true); + + ReservationListResponse response = null; + try { + response = clientService.listReservations(request); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + Assert.assertNotNull(response); + Assert.assertEquals(1, response.getReservationAllocationState().size()); + Assert.assertEquals(response.getReservationAllocationState().get(0) + .getReservationId().getId(), sRequest.getReservationId().getId()); + + // List reservations, search by invalid end time < -1. + request = ReservationListRequest + .newInstance(ReservationSystemTestUtil.reservationQ, "", 1, -10, true); response = null; try { @@ -1261,16 +1336,48 @@ public void testReservationAPIs() { } catch (Exception e) { Assert.fail(e.getMessage()); } + Assert.assertNotNull(response); + Assert.assertEquals(1, response.getReservationAllocationState().size()); + Assert.assertEquals(response.getReservationAllocationState().get(0) + .getReservationId().getId(), sRequest.getReservationId().getId()); + + rm.stop(); + } + + @Test + public void testListReservationsByTimeIntervalContainingNoReservations() { + ResourceManager rm = setupResourceManager(); + ClientRMService clientService = rm.getClientRMService(); + Clock clock = new UTCClock(); + long arrival = clock.getTime(); + long duration = 60000; + long deadline = (long) (arrival + 1.05 * duration); + ReservationSubmissionRequest sRequest = + submitReservationTestHelper(clientService, arrival, deadline, duration); + + // List reservations, search by very large start time. + ReservationListRequest request = ReservationListRequest.newInstance( + ReservationSystemTestUtil.reservationQ, "", Long.MAX_VALUE, -1, false); + + ReservationListResponse response = null; + try { + response = clientService.listReservations(request); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } // Ensure all reservations are filtered out. Assert.assertNotNull(response); - Assert.assertEquals(0, response.getReservationAllocationState().size()); + Assert.assertEquals(response.getReservationAllocationState().size(), 0); + + duration = 30000; + deadline = sRequest.getReservationDefinition().getDeadline(); // List reservations, search by start time after the reservation // end time. request = ReservationListRequest.newInstance( - ReservationSystemTestUtil.reservationQ, "", deadline + duration, - deadline + 2 * duration, false); + ReservationSystemTestUtil.reservationQ, "", deadline + duration, + deadline + 2 * duration, false); response = null; try { @@ -1283,11 +1390,12 @@ public void testReservationAPIs() { Assert.assertNotNull(response); Assert.assertEquals(response.getReservationAllocationState().size(), 0); - // List reservations, search by end time before the reservation start + arrival = clock.getTime(); + // List reservations, search by end time before the reservation start // time. request = ReservationListRequest.newInstance( - ReservationSystemTestUtil.reservationQ, "", 0, arrival - - duration, false); + ReservationSystemTestUtil.reservationQ, "", 0, arrival - duration, + false); response = null; try { @@ -1300,10 +1408,9 @@ public void testReservationAPIs() { Assert.assertNotNull(response); Assert.assertEquals(response.getReservationAllocationState().size(), 0); - // List reservations, search by a very small end time. - request = ReservationListRequest.newInstance( - ReservationSystemTestUtil.reservationQ, "", 0, 1, - false); + // List reservations, search by very small end time. + request = ReservationListRequest + .newInstance(ReservationSystemTestUtil.reservationQ, "", 0, 1, false); response = null; try { @@ -1316,6 +1423,21 @@ public void testReservationAPIs() { Assert.assertNotNull(response); Assert.assertEquals(response.getReservationAllocationState().size(), 0); + rm.stop(); + } + + @Test + public void testReservationDelete() { + ResourceManager rm = setupResourceManager(); + ClientRMService clientService = rm.getClientRMService(); + Clock clock = new UTCClock(); + long arrival = clock.getTime(); + long duration = 60000; + long deadline = (long) (arrival + 1.05 * duration); + ReservationSubmissionRequest sRequest = + submitReservationTestHelper(clientService, arrival, deadline, duration); + + ReservationId reservationID = sRequest.getReservationId(); // Delete the reservation ReservationDeleteRequest dRequest = ReservationDeleteRequest.newInstance(reservationID); @@ -1325,15 +1447,15 @@ public void testReservationAPIs() { } catch (Exception e) { Assert.fail(e.getMessage()); } - Assert.assertNotNull(sResponse); - LOG.info("Delete reservation response: " + dResponse); + Assert.assertNotNull(dResponse); + System.out.println("Delete reservation response: " + dResponse); // List reservations, search by non-existent reservationID - request = ReservationListRequest.newInstance( - ReservationSystemTestUtil.reservationQ, reservationID.toString(), - -1, -1, false); + ReservationListRequest request = ReservationListRequest.newInstance( + ReservationSystemTestUtil.reservationQ, reservationID.toString(), -1, + -1, false); - response = null; + ReservationListResponse response = null; try { response = clientService.listReservations(request); } catch (Exception e) { @@ -1342,10 +1464,7 @@ public void testReservationAPIs() { Assert.assertNotNull(response); Assert.assertEquals(0, response.getReservationAllocationState().size()); - // clean-up rm.stop(); - nm = null; - rm = null; } @Test diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestReservationSystemWithRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestReservationSystemWithRMHA.java index 9a0f2c91ac..c13d72de35 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestReservationSystemWithRMHA.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestReservationSystemWithRMHA.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.yarn.server.resourcemanager; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse; import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest; import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse; import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest; @@ -65,8 +67,11 @@ public void testSubmitReservationAndCheckAfterFailover() throws Exception { ClientRMService clientService = rm1.getClientRMService(); + ReservationId reservationID = getNewReservation(clientService) + .getReservationId(); // create a reservation - ReservationSubmissionRequest request = createReservationSubmissionRequest(); + ReservationSubmissionRequest request = createReservationSubmissionRequest( + reservationID); ReservationSubmissionResponse response = null; try { response = clientService.submitReservation(request); @@ -74,7 +79,6 @@ public void testSubmitReservationAndCheckAfterFailover() throws Exception { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - ReservationId reservationID = response.getReservationId(); Assert.assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); @@ -98,8 +102,12 @@ public void testUpdateReservationAndCheckAfterFailover() throws Exception { ClientRMService clientService = rm1.getClientRMService(); + ReservationId reservationID = getNewReservation(clientService) + .getReservationId(); + // create a reservation - ReservationSubmissionRequest request = createReservationSubmissionRequest(); + ReservationSubmissionRequest request = createReservationSubmissionRequest( + reservationID); ReservationSubmissionResponse response = null; try { response = clientService.submitReservation(request); @@ -107,7 +115,6 @@ public void testUpdateReservationAndCheckAfterFailover() throws Exception { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - ReservationId reservationID = response.getReservationId(); Assert.assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); ReservationDefinition reservationDefinition = @@ -144,8 +151,12 @@ public void testDeleteReservationAndCheckAfterFailover() throws Exception { ClientRMService clientService = rm1.getClientRMService(); + ReservationId reservationID = getNewReservation(clientService) + .getReservationId(); + // create a reservation - ReservationSubmissionRequest request = createReservationSubmissionRequest(); + ReservationSubmissionRequest request = createReservationSubmissionRequest( + reservationID); ReservationSubmissionResponse response = null; try { response = clientService.submitReservation(request); @@ -153,7 +164,6 @@ public void testDeleteReservationAndCheckAfterFailover() throws Exception { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - ReservationId reservationID = response.getReservationId(); Assert.assertNotNull(reservationID); // Delete the reservation @@ -199,13 +209,14 @@ private void addNodeCapacityToPlan(MockRM rm, int memory, int vCores) { } } - private ReservationSubmissionRequest createReservationSubmissionRequest() { + private ReservationSubmissionRequest createReservationSubmissionRequest( + ReservationId reservationId) { Clock clock = new UTCClock(); long arrival = clock.getTime(); long duration = 60000; long deadline = (long) (arrival + duration + 1500); - return ReservationSystemTestUtil.createSimpleReservationRequest(4, arrival, - deadline, duration); + return ReservationSystemTestUtil.createSimpleReservationRequest( + reservationId, 4, arrival, deadline, duration); } private void validateReservation(Plan plan, ReservationId resId, @@ -224,8 +235,12 @@ public void testSubmitReservationFailoverAndDelete() throws Exception { ClientRMService clientService = rm1.getClientRMService(); + ReservationId reservationID = getNewReservation(clientService) + .getReservationId(); + // create a reservation - ReservationSubmissionRequest request = createReservationSubmissionRequest(); + ReservationSubmissionRequest request = createReservationSubmissionRequest( + reservationID); ReservationSubmissionResponse response = null; try { response = clientService.submitReservation(request); @@ -233,7 +248,6 @@ public void testSubmitReservationFailoverAndDelete() throws Exception { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - ReservationId reservationID = response.getReservationId(); Assert.assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); ReservationDefinition reservationDefinition = @@ -273,10 +287,14 @@ public void testFailoverAndSubmitReservation() throws Exception { explicitFailover(); addNodeCapacityToPlan(rm2, 102400, 100); + ClientRMService clientService = rm2.getClientRMService(); + + ReservationId reservationID = getNewReservation(clientService) + .getReservationId(); // create a reservation - ClientRMService clientService = rm2.getClientRMService(); - ReservationSubmissionRequest request = createReservationSubmissionRequest(); + ReservationSubmissionRequest request = createReservationSubmissionRequest( + reservationID); ReservationSubmissionResponse response = null; try { response = clientService.submitReservation(request); @@ -284,7 +302,6 @@ public void testFailoverAndSubmitReservation() throws Exception { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - ReservationId reservationID = response.getReservationId(); Assert.assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); ReservationDefinition reservationDefinition = @@ -304,8 +321,12 @@ public void testSubmitReservationFailoverAndUpdate() throws Exception { ClientRMService clientService = rm1.getClientRMService(); + ReservationId reservationID = getNewReservation(clientService) + .getReservationId(); + // create a reservation - ReservationSubmissionRequest request = createReservationSubmissionRequest(); + ReservationSubmissionRequest request = createReservationSubmissionRequest( + reservationID); ReservationSubmissionResponse response = null; try { response = clientService.submitReservation(request); @@ -313,7 +334,6 @@ public void testSubmitReservationFailoverAndUpdate() throws Exception { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - ReservationId reservationID = response.getReservationId(); Assert.assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); ReservationDefinition reservationDefinition = @@ -353,8 +373,12 @@ public void testSubmitUpdateReservationFailoverAndDelete() throws Exception { ClientRMService clientService = rm1.getClientRMService(); + ReservationId reservationID = getNewReservation(clientService) + .getReservationId(); + // create a reservation - ReservationSubmissionRequest request = createReservationSubmissionRequest(); + ReservationSubmissionRequest request = createReservationSubmissionRequest( + reservationID); ReservationSubmissionResponse response = null; try { response = clientService.submitReservation(request); @@ -362,7 +386,6 @@ public void testSubmitUpdateReservationFailoverAndDelete() throws Exception { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - ReservationId reservationID = response.getReservationId(); Assert.assertNotNull(reservationID); LOG.info("Submit reservation response: " + reservationID); ReservationDefinition reservationDefinition = @@ -419,8 +442,12 @@ public void testReservationResizeAfterFailover() throws Exception { ClientRMService clientService = rm1.getClientRMService(); - // create 3 reservations - ReservationSubmissionRequest request = createReservationSubmissionRequest(); + ReservationId resID1 = getNewReservation(clientService) + .getReservationId(); + + // create a reservation + ReservationSubmissionRequest request = createReservationSubmissionRequest( + resID1); ReservationDefinition reservationDefinition = request.getReservationDefinition(); ReservationSubmissionResponse response = null; @@ -430,25 +457,30 @@ public void testReservationResizeAfterFailover() throws Exception { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - ReservationId resID1 = response.getReservationId(); Assert.assertNotNull(resID1); LOG.info("Submit reservation response: " + resID1); + + ReservationId resID2 = getNewReservation(clientService) + .getReservationId(); + request.setReservationId(resID2); try { response = clientService.submitReservation(request); } catch (Exception e) { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - ReservationId resID2 = response.getReservationId(); Assert.assertNotNull(resID2); LOG.info("Submit reservation response: " + resID2); + + ReservationId resID3 = getNewReservation(clientService) + .getReservationId(); + request.setReservationId(resID3); try { response = clientService.submitReservation(request); } catch (Exception e) { Assert.fail(e.getMessage()); } Assert.assertNotNull(response); - ReservationId resID3 = response.getReservationId(); Assert.assertNotNull(resID3); LOG.info("Submit reservation response: " + resID3); @@ -515,4 +547,18 @@ private void waitForReservationActivation(MockRM rm, } } + private GetNewReservationResponse getNewReservation(ClientRMService + clientRMService) { + GetNewReservationRequest newReservationRequest = GetNewReservationRequest + .newInstance(); + GetNewReservationResponse getNewReservationResponse = null; + try { + getNewReservationResponse = clientRMService.getNewReservation( + newReservationRequest); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + return getNewReservationResponse; + } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java index 4aef7aee90..9ea1044b01 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java @@ -197,7 +197,8 @@ public static ReservationDefinition createSimpleReservationDefinition( } public static ReservationSubmissionRequest createSimpleReservationRequest( - int numContainers, long arrival, long deadline, long duration) { + ReservationId reservationId, int numContainers, long arrival, + long deadline, long duration) { // create a request with a single atomic ask ReservationRequest r = ReservationRequest.newInstance(Resource.newInstance(1024, 1), @@ -210,7 +211,7 @@ public static ReservationSubmissionRequest createSimpleReservationRequest( "testClientRMService#reservation"); ReservationSubmissionRequest request = ReservationSubmissionRequest.newInstance(rDef, - reservationQ); + reservationQ, reservationId); return request; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java index a60cf17dc1..40d46d0ac0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesReservation.java @@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.io.File; @@ -102,6 +100,8 @@ public class TestRMWebServicesReservation extends JerseyTestBase { // This is what is used in the test resource files. private static final String DEFAULT_QUEUE = "dedicated"; private static final String LIST_RESERVATION_PATH = "reservation/list"; + private static final String GET_NEW_RESERVATION_PATH = + "reservation/new-reservation"; public static class GuiceServletConfig extends GuiceServletContextListener { @@ -330,24 +330,88 @@ public void tearDown() throws Exception { public void testSubmitReservation() throws Exception { rm.start(); setupCluster(100); - ReservationId rid = - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON); + + ReservationId rid = getReservationIdTestHelper(1); + ClientResponse response = reservationSubmissionTestHelper( + "reservation/submit", MediaType.APPLICATION_JSON, rid); if (this.isAuthenticationEnabled()) { - assertNotNull(rid); + assertTrue(isHttpSuccessResponse(response)); + verifyReservationCount(1); } rm.stop(); } + @Test + public void testSubmitDuplicateReservation() throws Exception { + rm.start(); + setupCluster(100); + + ReservationId rid = getReservationIdTestHelper(1); + long currentTimestamp = clock.getTime() + MINIMUM_RESOURCE_DURATION; + ClientResponse response = reservationSubmissionTestHelper( + "reservation/submit", MediaType.APPLICATION_JSON, currentTimestamp, "", + rid); + + // Make sure that the first submission is successful + if (this.isAuthenticationEnabled()) { + assertTrue(isHttpSuccessResponse(response)); + } + + response = reservationSubmissionTestHelper( + "reservation/submit", MediaType.APPLICATION_JSON, currentTimestamp, "", + rid); + + // Make sure that the second submission is successful + if (this.isAuthenticationEnabled()) { + assertTrue(isHttpSuccessResponse(response)); + verifyReservationCount(1); + } + + rm.stop(); + } + + @Test + public void testSubmitDifferentReservationWithSameId() throws Exception { + rm.start(); + setupCluster(100); + + ReservationId rid = getReservationIdTestHelper(1); + long currentTimestamp = clock.getTime() + MINIMUM_RESOURCE_DURATION; + ClientResponse response = reservationSubmissionTestHelper( + "reservation/submit", MediaType.APPLICATION_JSON, currentTimestamp, + "res1", rid); + + // Make sure that the first submission is successful + if (this.isAuthenticationEnabled()) { + assertTrue(isHttpSuccessResponse(response)); + } + + // Change the reservation definition. + response = reservationSubmissionTestHelper( + "reservation/submit", MediaType.APPLICATION_JSON, + currentTimestamp + MINIMUM_RESOURCE_DURATION, "res1", rid); + + // Make sure that the second submission is unsuccessful + if (this.isAuthenticationEnabled()) { + assertTrue(!isHttpSuccessResponse(response)); + verifyReservationCount(1); + } + + rm.stop(); + } + @Test public void testFailedSubmitReservation() throws Exception { rm.start(); // setup a cluster too small to accept the reservation setupCluster(1); - ReservationId rid = - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON); - assertNull(rid); + + ReservationId rid = getReservationIdTestHelper(1); + ClientResponse response = reservationSubmissionTestHelper( + "reservation/submit", MediaType.APPLICATION_JSON, rid); + + assertTrue(!isHttpSuccessResponse(response)); + rm.stop(); } @@ -355,13 +419,14 @@ public void testFailedSubmitReservation() throws Exception { public void testUpdateReservation() throws JSONException, Exception { rm.start(); setupCluster(100); - ReservationId rid = - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON); + + ReservationId rid = getReservationIdTestHelper(1); + ClientResponse response = reservationSubmissionTestHelper( + "reservation/submit", MediaType.APPLICATION_JSON, rid); if (this.isAuthenticationEnabled()) { - assertNotNull(rid); + assertTrue(isHttpSuccessResponse(response)); } - testUpdateReservationHelper("reservation/update", rid, + updateReservationTestHelper("reservation/update", rid, MediaType.APPLICATION_JSON); rm.stop(); @@ -373,11 +438,15 @@ public void testTimeIntervalRequestListReservation() throws Exception { setupCluster(100); long time = clock.getTime() + MINIMUM_RESOURCE_DURATION; - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, time, "res_1", 1); - testSubmissionReservationHelper("reservation/submit", + + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, time, "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION, - "res_2", 2); + "res_2", id2); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("start-time", Long.toString((long) (time * 0.9))) @@ -410,11 +479,19 @@ public void testSameTimeIntervalRequestListReservation() throws Exception { long time = clock.getTime() + MINIMUM_RESOURCE_DURATION; - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, time, "res_1", 1); - testSubmissionReservationHelper("reservation/submit", + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + + // If authentication is not enabled then id1 and id2 will be null + if (!this.isAuthenticationEnabled() && id1 == null && id2 == null) { + return; + } + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, time, "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION, - "res_2", 2); + "res_2", id2); String timeParam = Long.toString(time + MINIMUM_RESOURCE_DURATION / 2); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) @@ -447,11 +524,14 @@ public void testInvalidTimeIntervalRequestListReservation() throws long time = clock.getTime() + MINIMUM_RESOURCE_DURATION; - ReservationId res1 = testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, time, "res_1", 1); - ReservationId res2 = testSubmissionReservationHelper("reservation/submit", + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, time, "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION, - "res_2", 2); + "res_2", id2); WebResource resource; resource = constructWebResource(LIST_RESERVATION_PATH) @@ -483,11 +563,14 @@ public void testInvalidEndTimeRequestListReservation() throws Exception { long time = clock.getTime() + MINIMUM_RESOURCE_DURATION; - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, time, "res_1", 1); - testSubmissionReservationHelper("reservation/submit", + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, time, "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION, - "res_2", 2); + "res_2", id2); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("start-time", Long.toString((long) (time + @@ -520,11 +603,14 @@ public void testEmptyEndTimeRequestListReservation() throws Exception { long time = clock.getTime() + MINIMUM_RESOURCE_DURATION; - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, time, "res_1", 1); - testSubmissionReservationHelper("reservation/submit", + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, time, "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION, - "res_2", 2); + "res_2", id2); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("start-time", new Long((long) (time + @@ -556,11 +642,14 @@ public void testInvalidStartTimeRequestListReservation() throws Exception { long time = clock.getTime() + MINIMUM_RESOURCE_DURATION; - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, time, "res_1", 1); - testSubmissionReservationHelper("reservation/submit", + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, time, "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION, - "res_2", 2); + "res_2", id2); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("start-time", "-1") @@ -592,13 +681,16 @@ public void testEmptyStartTimeRequestListReservation() throws Exception { rm.start(); setupCluster(100); + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + long time = clock.getTime() + MINIMUM_RESOURCE_DURATION; - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, time, "res_1", 1); - testSubmissionReservationHelper("reservation/submit", + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, time, "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", MediaType.APPLICATION_JSON, time + MINIMUM_RESOURCE_DURATION, - "res_2", 2); + "res_2", id2); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("end-time", new Long((long)(time + @@ -629,10 +721,13 @@ public void testQueueOnlyRequestListReservation() throws Exception { rm.start(); setupCluster(100); - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1); - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_2", 2); + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_2", id2); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("queue", DEFAULT_QUEUE); @@ -656,10 +751,13 @@ public void testEmptyQueueRequestListReservation() throws Exception { rm.start(); setupCluster(100); - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1); - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_2", 2); + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_2", id2); WebResource resource = constructWebResource(LIST_RESERVATION_PATH); @@ -673,10 +771,13 @@ public void testNonExistentQueueRequestListReservation() throws Exception { rm.start(); setupCluster(100); - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1); - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_2", 2); + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_2", id2); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("queue", DEFAULT_QUEUE + "_invalid"); @@ -691,10 +792,15 @@ public void testReservationIdRequestListReservation() throws Exception { rm.start(); setupCluster(100); - ReservationId id1 = testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1); - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_2", 2); + ReservationId id1 = getReservationIdTestHelper(1); + ReservationId id2 = getReservationIdTestHelper(2); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_1", id1); + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_2", id2); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("include-resource-allocations", "true") @@ -726,8 +832,10 @@ public void testInvalidReservationIdRequestListReservation() throws rm.start(); setupCluster(100); - ReservationId id1 = testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1); + ReservationId id1 = getReservationIdTestHelper(1); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_1", id1); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("queue", DEFAULT_QUEUE); @@ -747,8 +855,9 @@ public void testIncludeResourceAllocations() throws Exception { rm.start(); setupCluster(100); - ReservationId id1 = testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1); + ReservationId id1 = getReservationIdTestHelper(1); + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_1", id1); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("include-resource-allocations", "true") @@ -781,8 +890,10 @@ public void testExcludeResourceAllocations() throws Exception { rm.start(); setupCluster(100); - ReservationId id1 = testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON, clock.getTime(), "res_1", 1); + ReservationId id1 = getReservationIdTestHelper(1); + + reservationSubmissionTestHelper("reservation/submit", + MediaType.APPLICATION_JSON, clock.getTime(), "res_1", id1); WebResource resource = constructWebResource(LIST_RESERVATION_PATH) .queryParam("include-resource-allocations", "false") @@ -818,40 +929,76 @@ public void testDeleteReservation() throws JSONException, Exception { rm.registerNode("127.0.0." + i + ":1234", 100 * 1024); amNodeManager.nodeHeartbeat(true); } - ReservationId rid = - testSubmissionReservationHelper("reservation/submit", - MediaType.APPLICATION_JSON); - if (this.isAuthenticationEnabled()) { - assertNotNull(rid); - } + + ReservationId rid = getReservationIdTestHelper(1); + + reservationSubmissionTestHelper("reservation/submit", MediaType + .APPLICATION_JSON, rid); testDeleteReservationHelper("reservation/delete", rid, MediaType.APPLICATION_JSON); rm.stop(); } - private ReservationId testSubmissionReservationHelper(String path, - String media) throws Exception { + /** + * This method is used when a ReservationId is required. Attempt to use REST + * API. If authentication is not enabled, ensure that the response status is + * unauthorized and generate a ReservationId because downstream components + * require a ReservationId for testing. + * @param fallbackReservationId the ReservationId to use if authentication + * is not enabled, causing the getNewReservation + * API to fail. + * @return the object representing the reservation ID. + */ + private ReservationId getReservationIdTestHelper(int fallbackReservationId) + throws Exception { + Thread.sleep(1000); + ClientResponse response = constructWebResource(GET_NEW_RESERVATION_PATH) + .type(MediaType.APPLICATION_JSON) + .accept(MediaType.APPLICATION_JSON) + .post(ClientResponse.class); + + if (!this.isAuthenticationEnabled()) { + assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus()); + return ReservationId.newInstance(clock.getTime(), fallbackReservationId); + } + + System.out.println("RESPONSE:" + response); + assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); + JSONObject json = response.getEntity(JSONObject.class); + + assertEquals("incorrect number of elements", 1, json.length()); + ReservationId rid = null; + try { + rid = ReservationId.parseReservationId(json.getString("reservation-id")); + } catch (JSONException j) { + // failure is possible and is checked outside + } + return rid; + } + + private ClientResponse reservationSubmissionTestHelper(String path, + String media, ReservationId reservationId) throws Exception { long arrival = clock.getTime() + MINIMUM_RESOURCE_DURATION; - return testSubmissionReservationHelper(path, media, arrival, "res_1", 1); + return reservationSubmissionTestHelper(path, media, arrival, "res_1", + reservationId); } - private ReservationId testSubmissionReservationHelper(String path, - String media, Long arrival, String reservationName, int expectedId) - throws Exception { + private ClientResponse reservationSubmissionTestHelper(String path, + String media, Long arrival, String reservationName, + ReservationId reservationId) throws Exception { String reservationJson = loadJsonFile("submit-reservation.json"); - String reservationJsonRequest = String.format(reservationJson, arrival, - arrival + MINIMUM_RESOURCE_DURATION, reservationName); + String reservationJsonRequest = String.format(reservationJson, + reservationId.toString(), arrival, arrival + MINIMUM_RESOURCE_DURATION, + reservationName); - return submitAndVerifyReservation(path, media, reservationJsonRequest, - expectedId); + return submitAndVerifyReservation(path, media, reservationJsonRequest); } - private ReservationId submitAndVerifyReservation(String path, String media, - String reservationJson, int expectedId) throws Exception { - + private ClientResponse submitAndVerifyReservation(String path, String media, + String reservationJson) throws Exception { JSONJAXBContext jc = new JSONJAXBContext(JSONConfiguration.mapped() .build(), ReservationSubmissionRequestInfo.class); @@ -867,25 +1014,12 @@ private ReservationId submitAndVerifyReservation(String path, String media, if (!this.isAuthenticationEnabled()) { assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus()); - return null; } - System.out.println("RESPONSE:" + response); - assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); - JSONObject json = response.getEntity(JSONObject.class); - - assertEquals("incorrect number of elements", 1, json.length()); - ReservationId rid = null; - try { - rid = ReservationId.parseReservationId(json.getString("reservation-id")); - assertEquals("incorrect return value", rid.getId(), expectedId); - } catch (JSONException j) { - // failure is possible and is checked outside - } - return rid; + return response; } - private void testUpdateReservationHelper(String path, + private void updateReservationTestHelper(String path, ReservationId reservationId, String media) throws JSONException, Exception { @@ -1002,6 +1136,25 @@ private JSONObject testListReservationHelper(WebResource resource, Status return response.getEntity(JSONObject.class); } + private void verifyReservationCount(int count) throws Exception { + WebResource resource = constructWebResource(LIST_RESERVATION_PATH) + .queryParam("queue", DEFAULT_QUEUE); + + JSONObject json = testListReservationHelper(resource); + + if (count == 1) { + // If there are any number other than one reservation, this will throw. + json.getJSONObject("reservations"); + } else { + JSONArray reservations = json.getJSONArray("reservations"); + assertTrue(reservations.length() == count); + } + } + + private boolean isHttpSuccessResponse(ClientResponse response) { + return (response.getStatus() / 100) == 2; + } + private void setupCluster(int nodes) throws Exception { for (int i = 0; i < nodes; i++) { MockNM amNodeManager = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json index 0a243a288d..580c5990aa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/submit-reservation.json @@ -1,5 +1,6 @@ { "queue" : "dedicated", + "reservation-id" : "%s", "reservation-definition" : { "arrival" : %s, "deadline" : %s, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ReservationSystem.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ReservationSystem.md index eda8d4d806..51d2b972f6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ReservationSystem.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ReservationSystem.md @@ -39,7 +39,9 @@ Flow of a Reservation With reference to the figure above, a typical reservation proceeds as follows: - * **Step 1** The user (or an automated tool on its behalf) submit a reservation request specified by the Reservation Definition Language (RDL). This describes the user need for resources over-time (e.g., a skyline of resources) and temporal constraints (e.g., deadline). This can be done both programmatically through the usual Client-to-RM protocols or via the REST api of the RM. + * **Step 0** The user (or an automated tool on its behalf) submits a reservation creation request, and receives a response containing the ReservationId. + + * **Step 1** The user (or an automated tool on its behalf) submits a reservation request specified by the Reservation Definition Language (RDL) and ReservationId retrieved from the previous step. This describes the user need for resources over-time (e.g., a skyline of resources) and temporal constraints (e.g., deadline). This can be done both programmatically through the usual Client-to-RM protocols or via the REST api of the RM. If a reservation is submitted with the same ReservationId, and the RDL is the same, a new reservation will not be created and the request will be successful. If the RDL is different, the reservation will be rejected, and the request will be unsuccessful. * **Step 2** The ReservationSystem leverages a ReservationAgent (GREE in the figure) to find a plausible allocation for the reservation in the Plan, a data structure tracking all reservation currently accepted and the available resources in the system. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md index dd6ac0448f..1b77127519 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md @@ -34,6 +34,7 @@ ResourceManager REST API's. * [Cluster Application Priority API](#Cluster_Application_Priority_API) * [Cluster Delegation Tokens API](#Cluster_Delegation_Tokens_API) * [Cluster Reservation API List](#Cluster_Reservation_API_List) +* [Cluster Reservation API Create](#Cluster_Reservation_API_Create) * [Cluster Reservation API Submit](#Cluster_Reservation_API_Submit) * [Cluster Reservation API Update](#Cluster_Reservation_API_Update) * [Cluster Reservation API Delete](#Cluster_Reservation_API_Delete) @@ -3385,10 +3386,82 @@ Response Body: ``` +Cluster Reservation API Create +--------------------------- + +Use the New Reservation API, to obtain a reservation-id which can then be used as part of the [Cluster Reservation API Submit](#Cluster_Reservation_API_Submit) to submit reservations. + +This feature is currently in the alpha stage and may change in the future. + +### URI + + * http://