You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-commits@hadoop.apache.org by vi...@apache.org on 2013/06/04 07:03:52 UTC

svn commit: r1489290 - in /hadoop/common/trunk/hadoop-yarn-project: ./ hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ h...

Author: vinodkv
Date: Tue Jun  4 05:03:51 2013
New Revision: 1489290

URL: http://svn.apache.org/r1489290
Log:
YARN-756. Move Preemption* records to yarn.api where they really belong. Contributed by Jian He.

Added:
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContainer.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContract.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/StrictPreemptionContract.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionContainerPBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionContractPBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionMessagePBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/StrictPreemptionContractPBImpl.java
Removed:
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/PreemptionContainer.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/PreemptionContract.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/PreemptionMessage.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StrictPreemptionContract.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/PreemptionContainerPBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/PreemptionContractPBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/PreemptionMessagePBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StrictPreemptionContractPBImpl.java
Modified:
    hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/PreemptionResourceRequest.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

Modified: hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt?rev=1489290&r1=1489289&r2=1489290&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt Tue Jun  4 05:03:51 2013
@@ -93,6 +93,9 @@ Release 2.1.0-beta - UNRELEASED
     YARN-635. Renamed YarnRemoteException to YarnException. (Siddharth Seth via
     vinodkv)
 
+    YARN-756. Move Preemption* records to yarn.api where they really belong.
+    (Jian He via vinodkv)
+
   NEW FEATURES
 
     YARN-482. FS: Extend SchedulingMode to intermediate queues. 

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java?rev=1489290&r1=1489289&r2=1489290&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java Tue Jun  4 05:03:51 2013
@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.api.AMRMPr
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.util.Records;

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/PreemptionResourceRequest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/PreemptionResourceRequest.java?rev=1489290&r1=1489289&r2=1489290&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/PreemptionResourceRequest.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/PreemptionResourceRequest.java Tue Jun  4 05:03:51 2013
@@ -21,6 +21,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.PreemptionContract;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 
 /**

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java?rev=1489290&r1=1489289&r2=1489290&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java Tue Jun  4 05:03:51 2013
@@ -25,12 +25,13 @@ import java.util.List;
 
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.PreemptionMessagePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl;

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContainer.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContainer.java?rev=1489290&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContainer.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContainer.java Tue Jun  4 05:03:51 2013
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Specific container requested back by the <code>ResourceManager</code>.
+ * @see PreemptionContract
+ * @see StrictPreemptionContract
+ */
+public abstract class PreemptionContainer {
+
+  public static PreemptionContainer newInstance(ContainerId id) {
+    PreemptionContainer container = Records.newRecord(PreemptionContainer.class);
+    container.setId(id);
+    return container;
+  }
+
+  /**
+   * @return Container referenced by this handle.
+   */
+  @Public
+  @Evolving
+  public abstract ContainerId getId();
+
+  @Private
+  @Unstable
+  public abstract void setId(ContainerId id);
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContract.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContract.java?rev=1489290&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContract.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContract.java Tue Jun  4 05:03:51 2013
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.PreemptionResourceRequest;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Description of resources requested back by the <code>ResourceManager</code>.
+ * The <code>ApplicationMaster</code> (AM) can satisfy this request according
+ * to its own priorities to prevent containers from being forcibly killed by
+ * the platform.
+ * @see PreemptionMessage
+ */
+public abstract class PreemptionContract {
+
+  public static PreemptionContract newInstance(
+      List<PreemptionResourceRequest> req, Set<PreemptionContainer> containers) {
+    PreemptionContract contract = Records.newRecord(PreemptionContract.class);
+    contract.setResourceRequest(req);
+    contract.setContainers(containers);
+    return contract;
+  }
+
+  /**
+   * If the AM releases resources matching these requests, then the {@link
+   * PreemptionContainer}s enumerated in {@link #getContainers()} should not be
+   * evicted from the cluster. Due to delays in propagating cluster state and
+   * sending these messages, there are conditions where satisfied contracts may
+   * not prevent the platform from killing containers.
+   * @return List of {@link PreemptionResourceRequest} to update the
+   * <code>ApplicationMaster</code> about resources requested back by the
+   * <code>ResourceManager</code>.
+   * @see AllocateRequest#setAskList(List)
+   */
+  @Public
+  @Evolving
+  public abstract List<PreemptionResourceRequest> getResourceRequest();
+
+  @Private
+  @Unstable
+  public abstract void setResourceRequest(List<PreemptionResourceRequest> req);
+
+  /**
+   * Assign the set of {@link PreemptionContainer} specifying which containers
+   * owned by the <code>ApplicationMaster</code> that may be reclaimed by the
+   * <code>ResourceManager</code>. If the AM prefers a different set of
+   * containers, then it may checkpoint or kill containers matching the
+   * description in {@link #getResourceRequest}.
+   * @return Set of containers at risk if the contract is not met.
+   */
+  @Public
+  @Evolving
+  public abstract Set<PreemptionContainer> getContainers();
+
+
+  @Private
+  @Unstable
+  public abstract void setContainers(Set<PreemptionContainer> containers);
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java?rev=1489290&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java Tue Jun  4 05:03:51 2013
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * A {@link PreemptionMessage} is part of the RM-AM protocol, and it is used by
+ * the RM to specify resources that the RM wants to reclaim from this
+ * <code>ApplicationMaster</code> (AM). The AM receives a {@link
+ * StrictPreemptionContract} message encoding which containers the platform may
+ * forcibly kill, granting it an opportunity to checkpoint state or adjust its
+ * execution plan. The message may also include a {@link PreemptionContract}
+ * granting the AM more latitude in selecting which resources to return to the
+ * cluster.
+ *
+ * The AM should decode both parts of the message. The {@link
+ * StrictPreemptionContract} specifies particular allocations that the RM
+ * requires back. The AM can checkpoint containers' state, adjust its execution
+ * plan to move the computation, or take no action and hope that conditions that
+ * caused the RM to ask for the container will change.
+ *
+ * In contrast, the {@link PreemptionContract} also includes a description of
+ * resources with a set of containers. If the AM releases containers matching
+ * that profile, then the containers enumerated in {@link
+ * PreemptionContract#getContainers()} may not be killed.
+ *
+ * Each preemption message reflects the RM's current understanding of the
+ * cluster state, so a request to return <emph>N</emph> containers may not
+ * reflect containers the AM is releasing, recently exited containers the RM has
+ * yet to learn about, or new containers allocated before the message was
+ * generated. Conversely, an RM may request a different profile of containers in
+ * subsequent requests.
+ *
+ * The policy enforced by the RM is part of the scheduler. Generally, only
+ * containers that have been requested consistently should be killed, but the
+ * details are not specified.
+ */
+@Public
+@Evolving
+public abstract class PreemptionMessage {
+
+  public static PreemptionMessage newInstance(StrictPreemptionContract set,
+      PreemptionContract contract) {
+    PreemptionMessage message = Records.newRecord(PreemptionMessage.class);
+    message.setStrictContract(set);
+    message.setContract(contract);
+    return message;
+  }
+
+  /**
+   * @return Specific resources that may be killed by the
+   * <code>ResourceManager</code>
+   */
+  @Public
+  @Evolving
+  public abstract StrictPreemptionContract getStrictContract();
+
+  @Private
+  @Unstable
+  public abstract void setStrictContract(StrictPreemptionContract set);
+
+  /**
+   * @return Contract describing resources to return to the cluster.
+   */
+  @Public
+  @Evolving
+  public abstract PreemptionContract getContract();
+
+  @Private
+  @Unstable
+  public abstract void setContract(PreemptionContract contract);
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/StrictPreemptionContract.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/StrictPreemptionContract.java?rev=1489290&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/StrictPreemptionContract.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/StrictPreemptionContract.java Tue Jun  4 05:03:51 2013
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records;
+
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Enumeration of particular allocations to be reclaimed. The platform will
+ * reclaim exactly these resources, so the <code>ApplicationMaster</code> (AM)
+ * may attempt to checkpoint work or adjust its execution plan to accommodate
+ * it. In contrast to {@link PreemptionContract}, the AM has no flexibility in
+ * selecting which resources to return to the cluster.
+ * @see PreemptionMessage
+ */
+@Public
+@Evolving
+public abstract class StrictPreemptionContract {
+
+  public static StrictPreemptionContract newInstance(Set<PreemptionContainer> containers) {
+    StrictPreemptionContract contract =
+        Records.newRecord(StrictPreemptionContract.class);
+    contract.setContainers(containers);
+    return contract;
+  }
+
+  /**
+   * Get the set of {@link PreemptionContainer} specifying containers owned by
+   * the <code>ApplicationMaster</code> that may be reclaimed by the
+   * <code>ResourceManager</code>.
+   * @return the set of {@link ContainerId} to be preempted.
+   */
+  @Public
+  @Evolving
+  public abstract Set<PreemptionContainer> getContainers();
+
+  @Private
+  @Unstable
+  public abstract void setContainers(Set<PreemptionContainer> containers);
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionContainerPBImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionContainerPBImpl.java?rev=1489290&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionContainerPBImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionContainerPBImpl.java Tue Jun  4 05:03:51 2013
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.impl;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.PreemptionContainer;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionContainerProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionContainerProtoOrBuilder;
+
+public class PreemptionContainerPBImpl extends PreemptionContainer {
+
+  PreemptionContainerProto proto =
+    PreemptionContainerProto.getDefaultInstance();
+  PreemptionContainerProto.Builder builder = null;
+
+  boolean viaProto = false;
+  private ContainerId id;
+
+  public PreemptionContainerPBImpl() {
+    builder = PreemptionContainerProto.newBuilder();
+  }
+
+  public PreemptionContainerPBImpl(PreemptionContainerProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public synchronized PreemptionContainerProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (id != null) {
+      builder.setId(convertToProtoFormat(id));
+    }
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = PreemptionContainerProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public synchronized ContainerId getId() {
+    PreemptionContainerProtoOrBuilder p = viaProto ? proto : builder;
+    if (id != null) {
+      return id;
+    }
+    if (!p.hasId()) {
+      return null;
+    }
+    id = convertFromProtoFormat(p.getId());
+    return id;
+  }
+
+  @Override
+  public synchronized void setId(final ContainerId id) {
+    maybeInitBuilder();
+    if (null == id) {
+      builder.clearId();
+    }
+    this.id = id;
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl)t).getProto();
+  }
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionContractPBImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionContractPBImpl.java?rev=1489290&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionContractPBImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionContractPBImpl.java Tue Jun  4 05:03:51 2013
@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.impl;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.protocolrecords.PreemptionResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.PreemptionResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.PreemptionContainer;
+import org.apache.hadoop.yarn.api.records.PreemptionContract;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionContainerProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionContractProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionContractProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionResourceRequestProto;
+
+public class PreemptionContractPBImpl extends PreemptionContract {
+
+  PreemptionContractProto proto = PreemptionContractProto.getDefaultInstance();
+  PreemptionContractProto.Builder builder = null;
+
+  boolean viaProto = false;
+  private Set<PreemptionContainer> containers;
+  private List<PreemptionResourceRequest> resources;
+
+  public PreemptionContractPBImpl() {
+    builder = PreemptionContractProto.newBuilder();
+  }
+
+  public PreemptionContractPBImpl(PreemptionContractProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public synchronized PreemptionContractProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.resources != null) {
+      addResourcesToProto();
+    }
+    if (this.containers != null) {
+      addContainersToProto();
+    }
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = PreemptionContractProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public synchronized Set<PreemptionContainer> getContainers() {
+    initPreemptionContainers();
+    return containers;
+  }
+
+  @Override
+  public synchronized void setContainers(
+      final Set<PreemptionContainer> containers) {
+    if (null == containers) {
+      builder.clearContainer();
+    }
+    this.containers = containers;
+  }
+
+  @Override
+  public synchronized List<PreemptionResourceRequest> getResourceRequest() {
+    initPreemptionResourceRequests();
+    return resources;
+  }
+
+  @Override
+  public synchronized void setResourceRequest(
+      final List<PreemptionResourceRequest> req) {
+    if (null == resources) {
+      builder.clearResource();
+    }
+    this.resources = req;
+  }
+
+  private void initPreemptionResourceRequests() {
+    if (resources != null) {
+      return;
+    }
+    PreemptionContractProtoOrBuilder p = viaProto ? proto : builder;
+    List<PreemptionResourceRequestProto> list = p.getResourceList();
+    resources = new ArrayList<PreemptionResourceRequest>();
+
+    for (PreemptionResourceRequestProto rr : list) {
+      resources.add(convertFromProtoFormat(rr));
+    }
+  }
+
+  private void addResourcesToProto() {
+    maybeInitBuilder();
+    builder.clearResource();
+    if (null == resources) {
+      return;
+    }
+    Iterable<PreemptionResourceRequestProto> iterable =
+      new Iterable<PreemptionResourceRequestProto>() {
+      @Override
+      public Iterator<PreemptionResourceRequestProto> iterator() {
+        return new Iterator<PreemptionResourceRequestProto>() {
+
+          Iterator<PreemptionResourceRequest> iter = resources.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public PreemptionResourceRequestProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+
+          }
+        };
+
+      }
+    };
+    builder.addAllResource(iterable);
+  }
+
+  private void initPreemptionContainers() {
+    if (containers != null) {
+      return;
+    }
+    PreemptionContractProtoOrBuilder p = viaProto ? proto : builder;
+    List<PreemptionContainerProto> list = p.getContainerList();
+    containers = new HashSet<PreemptionContainer>();
+
+    for (PreemptionContainerProto c : list) {
+      containers.add(convertFromProtoFormat(c));
+    }
+  }
+
+  private void addContainersToProto() {
+    maybeInitBuilder();
+    builder.clearContainer();
+    if (null == containers) {
+      return;
+    }
+    Iterable<PreemptionContainerProto> iterable =
+      new Iterable<PreemptionContainerProto>() {
+      @Override
+      public Iterator<PreemptionContainerProto> iterator() {
+        return new Iterator<PreemptionContainerProto>() {
+
+          Iterator<PreemptionContainer> iter = containers.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public PreemptionContainerProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+
+          }
+        };
+
+      }
+    };
+    builder.addAllContainer(iterable);
+  }
+
+  private PreemptionContainerPBImpl convertFromProtoFormat(PreemptionContainerProto p) {
+    return new PreemptionContainerPBImpl(p);
+  }
+
+  private PreemptionContainerProto convertToProtoFormat(PreemptionContainer t) {
+    return ((PreemptionContainerPBImpl)t).getProto();
+  }
+
+  private PreemptionResourceRequestPBImpl convertFromProtoFormat(PreemptionResourceRequestProto p) {
+    return new PreemptionResourceRequestPBImpl(p);
+  }
+
+  private PreemptionResourceRequestProto convertToProtoFormat(PreemptionResourceRequest t) {
+    return ((PreemptionResourceRequestPBImpl)t).getProto();
+  }
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionMessagePBImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionMessagePBImpl.java?rev=1489290&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionMessagePBImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/PreemptionMessagePBImpl.java Tue Jun  4 05:03:51 2013
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.impl;
+
+import org.apache.hadoop.yarn.api.records.PreemptionContract;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
+import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionContractProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionMessageProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionMessageProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StrictPreemptionContractProto;
+
+public class PreemptionMessagePBImpl extends PreemptionMessage {
+
+  PreemptionMessageProto proto = PreemptionMessageProto.getDefaultInstance();
+  PreemptionMessageProto.Builder builder = null;
+
+  boolean viaProto = false;
+  private StrictPreemptionContract strict;
+  private PreemptionContract contract;
+
+  public PreemptionMessagePBImpl() {
+    builder = PreemptionMessageProto.newBuilder();
+  }
+
+  public PreemptionMessagePBImpl(PreemptionMessageProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public synchronized PreemptionMessageProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (strict != null) {
+      builder.setStrictContract(convertToProtoFormat(strict));
+    }
+    if (contract != null) {
+      builder.setContract(convertToProtoFormat(contract));
+    }
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = PreemptionMessageProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public synchronized StrictPreemptionContract getStrictContract() {
+    PreemptionMessageProtoOrBuilder p = viaProto ? proto : builder;
+    if (strict != null) {
+      return strict;
+    }
+    if (!p.hasStrictContract()) {
+      return null;
+    }
+    strict = convertFromProtoFormat(p.getStrictContract());
+    return strict;
+  }
+
+  @Override
+  public synchronized void setStrictContract(StrictPreemptionContract strict) {
+    maybeInitBuilder();
+    if (null == strict) {
+      builder.clearStrictContract();
+    }
+    this.strict = strict;
+  }
+
+  @Override
+  public synchronized PreemptionContract getContract() {
+    PreemptionMessageProtoOrBuilder p = viaProto ? proto : builder;
+    if (contract != null) {
+      return contract;
+    }
+    if (!p.hasContract()) {
+      return null;
+    }
+    contract = convertFromProtoFormat(p.getContract());
+    return contract;
+  }
+
+  @Override
+  public synchronized void setContract(final PreemptionContract c) {
+    maybeInitBuilder();
+    if (null == c) {
+      builder.clearContract();
+    }
+    this.contract = c;
+  }
+
+  private StrictPreemptionContractPBImpl convertFromProtoFormat(
+      StrictPreemptionContractProto p) {
+    return new StrictPreemptionContractPBImpl(p);
+  }
+
+  private StrictPreemptionContractProto convertToProtoFormat(
+      StrictPreemptionContract t) {
+    return ((StrictPreemptionContractPBImpl)t).getProto();
+  }
+
+  private PreemptionContractPBImpl convertFromProtoFormat(
+      PreemptionContractProto p) {
+    return new PreemptionContractPBImpl(p);
+  }
+
+  private PreemptionContractProto convertToProtoFormat(
+      PreemptionContract t) {
+    return ((PreemptionContractPBImpl)t).getProto();
+  }
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/StrictPreemptionContractPBImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/StrictPreemptionContractPBImpl.java?rev=1489290&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/StrictPreemptionContractPBImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/StrictPreemptionContractPBImpl.java Tue Jun  4 05:03:51 2013
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.impl;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.PreemptionContainer;
+import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionContainerProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StrictPreemptionContractProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StrictPreemptionContractProtoOrBuilder;
+
+public class StrictPreemptionContractPBImpl extends StrictPreemptionContract {
+
+  StrictPreemptionContractProto proto =
+    StrictPreemptionContractProto.getDefaultInstance();
+  StrictPreemptionContractProto.Builder builder = null;
+
+  boolean viaProto = false;
+  private Set<PreemptionContainer> containers;
+
+  public StrictPreemptionContractPBImpl() {
+    builder = StrictPreemptionContractProto.newBuilder();
+  }
+
+  public StrictPreemptionContractPBImpl(StrictPreemptionContractProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public synchronized StrictPreemptionContractProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.containers != null) {
+      addContainersToProto();
+    }
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = StrictPreemptionContractProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public synchronized Set<PreemptionContainer> getContainers() {
+    initIds();
+    return containers;
+  }
+
+  @Override
+  public synchronized void setContainers(
+      final Set<PreemptionContainer> containers) {
+    if (null == containers) {
+      builder.clearContainer();
+    }
+    this.containers = containers;
+  }
+
+  private void initIds() {
+    if (containers != null) {
+      return;
+    }
+    StrictPreemptionContractProtoOrBuilder p = viaProto ? proto : builder;
+    List<PreemptionContainerProto> list = p.getContainerList();
+    containers = new HashSet<PreemptionContainer>();
+
+    for (PreemptionContainerProto c : list) {
+      containers.add(convertFromProtoFormat(c));
+    }
+  }
+
+  private void addContainersToProto() {
+    maybeInitBuilder();
+    builder.clearContainer();
+    if (containers == null) {
+      return;
+    }
+    Iterable<PreemptionContainerProto> iterable = new Iterable<PreemptionContainerProto>() {
+      @Override
+      public Iterator<PreemptionContainerProto> iterator() {
+        return new Iterator<PreemptionContainerProto>() {
+
+          Iterator<PreemptionContainer> iter = containers.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public PreemptionContainerProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+
+          }
+        };
+
+      }
+    };
+    builder.addAllContainer(iterable);
+  }
+
+  private PreemptionContainerPBImpl convertFromProtoFormat(PreemptionContainerProto p) {
+    return new PreemptionContainerPBImpl(p);
+  }
+
+  private PreemptionContainerProto convertToProtoFormat(PreemptionContainer t) {
+    return ((PreemptionContainerPBImpl)t).getProto();
+  }
+
+}

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java?rev=1489290&r1=1489289&r2=1489290&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java Tue Jun  4 05:03:51 2013
@@ -32,7 +32,6 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -52,6 +51,7 @@ import org.apache.hadoop.yarn.api.record
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java?rev=1489290&r1=1489289&r2=1489290&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java Tue Jun  4 05:03:51 2013
@@ -41,19 +41,19 @@ import org.apache.hadoop.yarn.api.protoc
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.PreemptionContainer;
-import org.apache.hadoop.yarn.api.protocolrecords.PreemptionContract;
 import org.apache.hadoop.yarn.api.protocolrecords.PreemptionResourceRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StrictPreemptionContract;
-import org.apache.hadoop.yarn.api.protocolrecords.PreemptionMessage;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.PreemptionContainer;
+import org.apache.hadoop.yarn.api.records.PreemptionContract;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;