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 23:21:23 UTC

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

Author: vinodkv
Date: Tue Jun  4 21:21:09 2013
New Revision: 1489614

URL: http://svn.apache.org/r1489614
Log:
YARN-756. Remnants from moving 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/PreemptionResourceRequest.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContainerPBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionMessagePBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionResourceRequestPBImpl.java
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java
Removed:
    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/PreemptionResourceRequestPBImpl.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
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
    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/proto/yarn_protos.proto
    hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
    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/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=1489614&r1=1489613&r2=1489614&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 21:21:09 2013
@@ -31,19 +31,19 @@ import org.apache.hadoop.yarn.api.record
 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;
+import org.apache.hadoop.yarn.api.records.impl.pb.PreemptionMessagePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.PreemptionMessageProto;
 
     
 public class AllocateResponsePBImpl extends AllocateResponse {

Modified: 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=1489614&r1=1489613&r2=1489614&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContract.java (original)
+++ 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 21:21:09 2013
@@ -25,7 +25,6 @@ import org.apache.hadoop.classification.
 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;
 
 /**

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/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/records/PreemptionResourceRequest.java?rev=1489614&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionResourceRequest.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionResourceRequest.java Tue Jun  4 21:21:09 2013
@@ -0,0 +1,45 @@
+/**
+ * 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.api.protocolrecords.AllocateRequest;
+
+/**
+ * Description of resources requested back by the cluster.
+ * @see PreemptionContract
+ * @see AllocateRequest#setAskList(java.util.List)
+ */
+public interface PreemptionResourceRequest {
+
+  /**
+   * @return Resource described in this request, to be matched against running
+   * containers.
+   */
+  @Public
+  @Evolving
+  public ResourceRequest getResourceRequest();
+
+  @Private
+  @Unstable
+  public void setResourceRequest(ResourceRequest req);
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/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/pb/PreemptionContainerPBImpl.java?rev=1489614&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContainerPBImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContainerPBImpl.java Tue Jun  4 21:21:09 2013
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.PreemptionContainer;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContainerProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.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/pb/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/pb/PreemptionContractPBImpl.java?rev=1489614&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java Tue Jun  4 21:21:09 2013
@@ -0,0 +1,227 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.impl.pb;
+
+import java.util.ArrayList;
+import java.util.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.PreemptionContract;
+import org.apache.hadoop.yarn.api.records.PreemptionResourceRequest;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContainerProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContractProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContractProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.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/pb/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/pb/PreemptionMessagePBImpl.java?rev=1489614&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionMessagePBImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionMessagePBImpl.java Tue Jun  4 21:21:09 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.pb;
+
+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.YarnProtos.PreemptionContractProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.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/pb/PreemptionResourceRequestPBImpl.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/pb/PreemptionResourceRequestPBImpl.java?rev=1489614&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionResourceRequestPBImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionResourceRequestPBImpl.java Tue Jun  4 21:21:09 2013
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.PreemptionResourceRequest;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionResourceRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
+
+public class PreemptionResourceRequestPBImpl implements PreemptionResourceRequest {
+
+  PreemptionResourceRequestProto proto =
+    PreemptionResourceRequestProto.getDefaultInstance();
+  PreemptionResourceRequestProto.Builder builder = null;
+
+  boolean viaProto = false;
+  private ResourceRequest rr;
+
+  public PreemptionResourceRequestPBImpl() {
+    builder = PreemptionResourceRequestProto.newBuilder();
+  }
+
+  public PreemptionResourceRequestPBImpl(PreemptionResourceRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public synchronized PreemptionResourceRequestProto 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 (rr != null) {
+      builder.setResource(convertToProtoFormat(rr));
+    }
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = PreemptionResourceRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public synchronized ResourceRequest getResourceRequest() {
+    PreemptionResourceRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (rr != null) {
+      return rr;
+    }
+    if (!p.hasResource()) {
+      return null;
+    }
+    rr = convertFromProtoFormat(p.getResource());
+    return rr;
+  }
+
+  @Override
+  public synchronized void setResourceRequest(final ResourceRequest rr) {
+    maybeInitBuilder();
+    if (null == rr) {
+      builder.clearResource();
+    }
+    this.rr = rr;
+  }
+
+  private ResourceRequestPBImpl convertFromProtoFormat(ResourceRequestProto p) {
+    return new ResourceRequestPBImpl(p);
+  }
+
+  private ResourceRequestProto convertToProtoFormat(ResourceRequest t) {
+    return ((ResourceRequestPBImpl)t).getProto();
+  }
+
+}

Added: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/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/pb/StrictPreemptionContractPBImpl.java?rev=1489614&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java (added)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java Tue Jun  4 21:21:09 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.pb;
+
+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.YarnProtos.PreemptionContainerProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.StrictPreemptionContractProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.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-api/src/main/proto/yarn_protos.proto
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto?rev=1489614&r1=1489613&r2=1489614&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto Tue Jun  4 21:21:09 2013
@@ -207,6 +207,28 @@ message ResourceRequestProto {
   optional bool relax_locality = 5 [default = true];
 }
 
+message PreemptionMessageProto {
+  optional StrictPreemptionContractProto strictContract = 1;
+  optional PreemptionContractProto contract = 2;
+}
+
+message StrictPreemptionContractProto {
+  repeated PreemptionContainerProto container = 1;
+}
+
+message PreemptionContractProto {
+  repeated PreemptionResourceRequestProto resource = 1;
+  repeated PreemptionContainerProto container = 2;
+}
+
+message PreemptionContainerProto {
+  optional ContainerIdProto id = 1;
+}
+
+message PreemptionResourceRequestProto {
+  optional ResourceRequestProto resource = 1;
+}
+
 ////////////////////////////////////////////////////////////////////////
 ////// From client_RM_Protocol /////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////

Modified: hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto?rev=1489614&r1=1489613&r2=1489614&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto (original)
+++ hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto Tue Jun  4 21:21:09 2013
@@ -70,28 +70,6 @@ message AllocateResponseProto {
   repeated hadoop.common.TokenProto nm_tokens = 9;
 }
 
-message PreemptionMessageProto {
-  optional StrictPreemptionContractProto strictContract = 1;
-  optional PreemptionContractProto contract = 2;
-}
-
-message StrictPreemptionContractProto {
-  repeated PreemptionContainerProto container = 1;
-}
-
-message PreemptionContractProto {
-  repeated PreemptionResourceRequestProto resource = 1;
-  repeated PreemptionContainerProto container = 2;
-}
-
-message PreemptionContainerProto {
-  optional ContainerIdProto id = 1;
-}
-
-message PreemptionResourceRequestProto {
-  optional ResourceRequestProto resource = 1;
-}
-
 //////////////////////////////////////////////////////
 /////// client_RM_Protocol ///////////////////////////
 //////////////////////////////////////////////////////

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=1489614&r1=1489613&r2=1489614&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 21:21:09 2013
@@ -41,7 +41,6 @@ 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.PreemptionResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -51,6 +50,7 @@ import org.apache.hadoop.yarn.api.record
 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.PreemptionResourceRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;