You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by as...@apache.org on 2018/01/25 05:19:14 UTC

[08/31] hadoop git commit: YARN-6593. [API] Introduce Placement Constraint object. (Konstantinos Karanasos via wangda)

YARN-6593. [API] Introduce Placement Constraint object. (Konstantinos Karanasos via wangda)

Change-Id: Id00edb7185fdf01cce6e40f920cac3585f8cbe9c


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

Branch: refs/heads/YARN-6592
Commit: e464cbf79a24960b8afa64695599e4ed629cb002
Parents: 59828be
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Aug 3 14:03:55 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Wed Jan 24 21:17:06 2018 -0800

----------------------------------------------------------------------
 .../yarn/api/resource/PlacementConstraint.java  | 567 +++++++++++++++++++
 .../yarn/api/resource/PlacementConstraints.java | 286 ++++++++++
 .../hadoop/yarn/api/resource/package-info.java  |  23 +
 .../src/main/proto/yarn_protos.proto            |  55 ++
 .../api/resource/TestPlacementConstraints.java  | 106 ++++
 .../PlacementConstraintFromProtoConverter.java  | 116 ++++
 .../pb/PlacementConstraintToProtoConverter.java | 174 ++++++
 .../apache/hadoop/yarn/api/pb/package-info.java |  23 +
 .../yarn/api/records/impl/pb/ProtoUtils.java    |  27 +
 .../PlacementConstraintTransformations.java     | 209 +++++++
 .../hadoop/yarn/api/resource/package-info.java  |  23 +
 .../TestPlacementConstraintPBConversion.java    | 195 +++++++
 .../TestPlacementConstraintTransformations.java | 183 ++++++
 13 files changed, 1987 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
new file mode 100644
index 0000000..f0e3982
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
@@ -0,0 +1,567 @@
+/**
+ * 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.resource;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+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.Unstable;
+
+/**
+ * {@code PlacementConstraint} represents a placement constraint for a resource
+ * allocation.
+ */
+@Public
+@Unstable
+public class PlacementConstraint {
+
+  /**
+   * The constraint expression tree.
+   */
+  private AbstractConstraint constraintExpr;
+
+  public PlacementConstraint(AbstractConstraint constraintExpr) {
+    this.constraintExpr = constraintExpr;
+  }
+
+  /**
+   * Get the constraint expression of the placement constraint.
+   *
+   * @return the constraint expression
+   */
+  public AbstractConstraint getConstraintExpr() {
+    return constraintExpr;
+  }
+
+  /**
+   * Interface used to enable the elements of the constraint tree to be visited.
+   */
+  @Private
+  public interface Visitable {
+    /**
+     * Visitor pattern.
+     *
+     * @param visitor visitor to be used
+     * @param <T> defines the type that the visitor will use and the return type
+     *          of the accept.
+     * @return the result of visiting a given object.
+     */
+    <T> T accept(Visitor<T> visitor);
+
+  }
+
+  /**
+   * Visitor API for a constraint tree.
+   *
+   * @param <T> determines the return type of the visit methods.
+   */
+  @Private
+  public interface Visitor<T> {
+    T visit(SingleConstraint constraint);
+
+    T visit(TargetExpression target);
+
+    T visit(TargetConstraint constraint);
+
+    T visit(CardinalityConstraint constraint);
+
+    T visit(And constraint);
+
+    T visit(Or constraint);
+
+    T visit(DelayedOr constraint);
+
+    T visit(TimedPlacementConstraint constraint);
+  }
+
+  /**
+   * Abstract class that acts as the superclass of all placement constraint
+   * classes.
+   */
+  public abstract static class AbstractConstraint implements Visitable {
+    public PlacementConstraint build() {
+      return new PlacementConstraint(this);
+    }
+  }
+
+  static final String NODE_SCOPE = "node";
+  static final String RACK_SCOPE = "rack";
+
+  /**
+   * Consider a set of nodes N that belongs to the scope specified in the
+   * constraint. If the target expressions are satisfied at least minCardinality
+   * times and at most max-cardinality times in the node set N, then the
+   * constraint is satisfied.
+   *
+   * For example, a constraint of the form {@code {RACK, 2, 10,
+   * allocationTag("zk")}}, requires an allocation to be placed within a rack
+   * that has at least 2 and at most 10 other allocations with tag "zk".
+   */
+  public static class SingleConstraint extends AbstractConstraint {
+    private String scope;
+    private int minCardinality;
+    private int maxCardinality;
+    private Set<TargetExpression> targetExpressions;
+
+    public SingleConstraint(String scope, int minCardinality,
+        int maxCardinality, Set<TargetExpression> targetExpressions) {
+      this.scope = scope;
+      this.minCardinality = minCardinality;
+      this.maxCardinality = maxCardinality;
+      this.targetExpressions = targetExpressions;
+    }
+
+    public SingleConstraint(String scope, int minC, int maxC,
+        TargetExpression... targetExpressions) {
+      this(scope, minC, maxC, new HashSet<>(Arrays.asList(targetExpressions)));
+    }
+
+    /**
+     * Get the scope of the constraint.
+     *
+     * @return the scope of the constraint
+     */
+    public String getScope() {
+      return scope;
+    }
+
+    /**
+     * Get the minimum cardinality of the constraint.
+     *
+     * @return the minimum cardinality of the constraint
+     */
+    public int getMinCardinality() {
+      return minCardinality;
+    }
+
+    /**
+     * Get the maximum cardinality of the constraint.
+     *
+     * @return the maximum cardinality of the constraint
+     */
+    public int getMaxCardinality() {
+      return maxCardinality;
+    }
+
+    /**
+     * Get the target expressions of the constraint.
+     *
+     * @return the set of target expressions
+     */
+    public Set<TargetExpression> getTargetExpressions() {
+      return targetExpressions;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class representing the target expressions that are used in placement
+   * constraints. They might refer to expressions on node attributes, allocation
+   * tags, or be self-targets (referring to the allocation to which the
+   * constraint is attached).
+   */
+  public static class TargetExpression implements Visitable {
+    /**
+     * Enum specifying the type of the target expression.
+     */
+    public enum TargetType {
+      NODE_ATTRIBUTE, ALLOCATION_TAG, SELF
+    }
+
+    private TargetType targetType;
+    private String targetKey;
+    private Set<String> targetValues;
+
+    public TargetExpression(TargetType targetType, String targetKey,
+        Set<String> targetValues) {
+      this.targetType = targetType;
+      this.targetKey = targetKey;
+      this.targetValues = targetValues;
+    }
+
+    public TargetExpression(TargetType targetType) {
+      this(targetType, null, new HashSet<>());
+    }
+
+    public TargetExpression(TargetType targetType, String targetKey,
+        String... targetValues) {
+      this(targetType, targetKey, new HashSet<>(Arrays.asList(targetValues)));
+    }
+
+    /**
+     * Get the type of the target expression.
+     *
+     * @return the type of the target expression
+     */
+    public TargetType getTargetType() {
+      return targetType;
+    }
+
+    /**
+     * Get the key of the target expression.
+     *
+     * @return the key of the target expression
+     */
+    public String getTargetKey() {
+      return targetKey;
+    }
+
+    /**
+     * Get the set of values of the target expression.
+     *
+     * @return the set of values of the target expression
+     */
+    public Set<String> getTargetValues() {
+      return targetValues;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = targetType != null ? targetType.hashCode() : 0;
+      result = 31 * result + (targetKey != null ? targetKey.hashCode() : 0);
+      result =
+          31 * result + (targetValues != null ? targetValues.hashCode() : 0);
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null) {
+        return false;
+      }
+      if (!(o instanceof TargetExpression)) {
+        return false;
+      }
+
+      TargetExpression that = (TargetExpression) o;
+      if (targetType != that.targetType) {
+        return false;
+      }
+      if (targetKey != null ? !targetKey.equals(that.targetKey)
+          : that.targetKey != null) {
+        return false;
+      }
+      return targetValues != null ? targetValues.equals(that.targetValues)
+          : that.targetValues == null;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a target constraint. Such a constraint requires an
+   * allocation to be placed within a scope that satisfies some specified
+   * expressions on node attributes and allocation tags.
+   *
+   * It is a specialized version of the {@link SingleConstraint}, where the
+   * minimum and the maximum cardinalities take specific values based on the
+   * {@link TargetOperator} used.
+   */
+  public static class TargetConstraint extends AbstractConstraint {
+    enum TargetOperator {
+      IN, NOT_IN
+    }
+
+    private TargetOperator op;
+    private String scope;
+    private Set<TargetExpression> targetExpressions;
+
+    public TargetConstraint(TargetOperator op, String scope,
+        Set<TargetExpression> targetExpressions) {
+      this.op = op;
+      this.scope = scope;
+      this.targetExpressions = targetExpressions;
+    }
+
+    /**
+     * Get the target operator of the constraint.
+     *
+     * @return the target operator
+     */
+    public TargetOperator getOp() {
+      return op;
+    }
+
+    /**
+     * Get the scope of the constraint.
+     *
+     * @return the scope of the constraint
+     */
+    public String getScope() {
+      return scope;
+    }
+
+    /**
+     * Get the set of target expressions.
+     *
+     * @return the set of target expressions
+     */
+    public Set<TargetExpression> getTargetExpressions() {
+      return targetExpressions;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a cardinality constraint. Such a constraint the
+   * number of allocations within a given scope to some minimum and maximum
+   * values.
+   *
+   * It is a specialized version of the {@link SingleConstraint}, where the
+   * target is self (i.e., the allocation to which the constraint is attached).
+   */
+  public static class CardinalityConstraint extends AbstractConstraint {
+    private String scope;
+    private int minCardinality;
+    private int maxCardinality;
+
+    public CardinalityConstraint(String scope, int minCardinality,
+        int maxCardinality) {
+      this.scope = scope;
+      this.minCardinality = minCardinality;
+      this.maxCardinality = maxCardinality;
+    }
+
+    /**
+     * Get the scope of the constraint.
+     *
+     * @return the scope of the constraint
+     */
+    public String getScope() {
+      return scope;
+    }
+
+    /**
+     * Get the minimum cardinality of the constraint.
+     *
+     * @return the minimum cardinality of the constraint
+     */
+    public int getMinCardinality() {
+      return minCardinality;
+    }
+
+    /**
+     * Get the maximum cardinality of the constraint.
+     *
+     * @return the maximum cardinality of the constraint
+     */
+    public int getMaxCardinality() {
+      return maxCardinality;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents composite constraints, which comprise other
+   * constraints, forming a constraint tree.
+   *
+   * @param <R> the type of constraints that are used as children of the
+   *          specific composite constraint
+   */
+  public abstract static class CompositeConstraint<R extends Visitable>
+      extends AbstractConstraint {
+
+    /**
+     * Get the children of this composite constraint.
+     *
+     * @return the children of the composite constraint
+     */
+    public abstract List<R> getChildren();
+  }
+
+  /**
+   * Class that represents a composite constraint that is a conjunction of other
+   * constraints.
+   */
+  public static class And extends CompositeConstraint<AbstractConstraint> {
+    private List<AbstractConstraint> children;
+
+    public And(List<AbstractConstraint> children) {
+      this.children = children;
+    }
+
+    public And(AbstractConstraint... children) {
+      this(Arrays.asList(children));
+    }
+
+    @Override
+    public List<AbstractConstraint> getChildren() {
+      return children;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a composite constraint that is a disjunction of other
+   * constraints.
+   */
+  public static class Or extends CompositeConstraint<AbstractConstraint> {
+    private List<AbstractConstraint> children;
+
+    public Or(List<AbstractConstraint> children) {
+      this.children = children;
+    }
+
+    public Or(AbstractConstraint... children) {
+      this(Arrays.asList(children));
+    }
+
+    @Override
+    public List<AbstractConstraint> getChildren() {
+      return children;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Class that represents a composite constraint that comprises a list of timed
+   * placement constraints (see {@link TimedPlacementConstraint}). The scheduler
+   * should try to satisfy first the first timed child constraint within the
+   * specified time window. If this is not possible, it should attempt to
+   * satisfy the second, and so on.
+   */
+  public static class DelayedOr
+      extends CompositeConstraint<TimedPlacementConstraint> {
+    private List<TimedPlacementConstraint> children = new ArrayList<>();
+
+    public DelayedOr(List<TimedPlacementConstraint> children) {
+      this.children = children;
+    }
+
+    public DelayedOr(TimedPlacementConstraint... children) {
+      this(Arrays.asList(children));
+    }
+
+    @Override
+    public List<TimedPlacementConstraint> getChildren() {
+      return children;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Represents a timed placement constraint that has to be satisfied within a
+   * time window.
+   */
+  public static class TimedPlacementConstraint implements Visitable {
+    /**
+     * The unit of scheduling delay.
+     */
+    public enum DelayUnit {
+      MILLISECONDS, OPPORTUNITIES
+    }
+
+    private AbstractConstraint constraint;
+    private long schedulingDelay;
+    private DelayUnit delayUnit;
+
+    public TimedPlacementConstraint(AbstractConstraint constraint,
+        long schedulingDelay, DelayUnit delayUnit) {
+      this.constraint = constraint;
+      this.schedulingDelay = schedulingDelay;
+      this.delayUnit = delayUnit;
+    }
+
+    public TimedPlacementConstraint(AbstractConstraint constraint,
+        long schedulingDelay) {
+      this(constraint, schedulingDelay, DelayUnit.MILLISECONDS);
+    }
+
+    public TimedPlacementConstraint(AbstractConstraint constraint) {
+      this(constraint, Long.MAX_VALUE, DelayUnit.MILLISECONDS);
+    }
+
+    /**
+     * Get the constraint that has to be satisfied within the time window.
+     *
+     * @return the constraint to be satisfied
+     */
+    public AbstractConstraint getConstraint() {
+      return constraint;
+    }
+
+    /**
+     * Sets the constraint that has to be satisfied within the time window.
+     *
+     * @param constraint the constraint to be satisfied
+     */
+    public void setConstraint(AbstractConstraint constraint) {
+      this.constraint = constraint;
+    }
+
+    /**
+     * Get the scheduling delay value that determines the time window within
+     * which the constraint has to be satisfied.
+     *
+     * @return the value of the scheduling delay
+     */
+    public long getSchedulingDelay() {
+      return schedulingDelay;
+    }
+
+    /**
+     * The unit of the scheduling delay.
+     *
+     * @return the unit of the delay
+     */
+    public DelayUnit getDelayUnit() {
+      return delayUnit;
+    }
+
+    @Override
+    public <T> T accept(Visitor<T> visitor) {
+      return visitor.visit(this);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
new file mode 100644
index 0000000..8e84280
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
@@ -0,0 +1,286 @@
+/**
+ * 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.resource;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression.TargetType;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+
+/**
+ * This class contains various static methods for the applications to create
+ * placement constraints (see also {@link PlacementConstraint}).
+ */
+@Public
+@Unstable
+public final class PlacementConstraints {
+
+  // Suppresses default constructor, ensuring non-instantiability.
+  private PlacementConstraints() {
+  }
+
+  // Creation of simple constraints.
+
+  public static final String NODE = PlacementConstraint.NODE_SCOPE;
+  public static final String RACK = PlacementConstraint.RACK_SCOPE;
+
+  /**
+   * Creates a constraint that requires allocations to be placed on nodes that
+   * satisfy all target expressions within the given scope (e.g., node or rack).
+   *
+   * For example, {@code targetIn(RACK, allocationTag("hbase-m"))}, allows
+   * allocations on nodes that belong to a rack that has at least one tag with
+   * value "hbase-m".
+   *
+   * @param scope the scope within which the target expressions should be
+   *          satisfied
+   * @param targetExpressions the expressions that need to be satisfied within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint targetIn(String scope,
+      TargetExpression... targetExpressions) {
+    return new SingleConstraint(scope, 1, Integer.MAX_VALUE, targetExpressions);
+  }
+
+  /**
+   * Creates a constraint that requires allocations to be placed on nodes that
+   * belong to a scope (e.g., node or rack) that does not satisfy any of the
+   * target expressions.
+   *
+   * @param scope the scope within which the target expressions should not be
+   *          true
+   * @param targetExpressions the expressions that need to not be true within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint targetNotIn(String scope,
+      TargetExpression... targetExpressions) {
+    return new SingleConstraint(scope, 0, 0, targetExpressions);
+  }
+
+  /**
+   * Creates a constraint that restricts the number of allocations within a
+   * given scope (e.g., node or rack).
+   *
+   * For example, {@code cardinality(NODE, 3, 10)}, restricts the number of
+   * allocations per node to be no less than 3 and no more than 10.
+   *
+   * @param scope the scope of the constraint
+   * @param minCardinality determines the minimum number of allocations within
+   *          the scope
+   * @param maxCardinality determines the maximum number of allocations within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint cardinality(String scope, int minCardinality,
+      int maxCardinality) {
+    return new SingleConstraint(scope, minCardinality, maxCardinality,
+        PlacementTargets.self());
+  }
+
+  /**
+   * Similar to {@link #cardinality(String, int, int)}, but determines only the
+   * minimum cardinality (the maximum cardinality is unbound).
+   *
+   * @param scope the scope of the constraint
+   * @param minCardinality determines the minimum number of allocations within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint minCardinality(String scope,
+      int minCardinality) {
+    return cardinality(scope, minCardinality, Integer.MAX_VALUE);
+  }
+
+  /**
+   * Similar to {@link #cardinality(String, int, int)}, but determines only the
+   * maximum cardinality (the minimum can be as low as 0).
+   *
+   * @param scope the scope of the constraint
+   * @param maxCardinality determines the maximum number of allocations within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint maxCardinality(String scope,
+      int maxCardinality) {
+    return cardinality(scope, 0, maxCardinality);
+  }
+
+  /**
+   * This constraint generalizes the cardinality and target constraints.
+   *
+   * Consider a set of nodes N that belongs to the scope specified in the
+   * constraint. If the target expressions are satisfied at least minCardinality
+   * times and at most max-cardinality times in the node set N, then the
+   * constraint is satisfied.
+   *
+   * For example, {@code targetCardinality(RACK, 2, 10, allocationTag("zk"))},
+   * requires an allocation to be placed within a rack that has at least 2 and
+   * at most 10 other allocations with tag "zk".
+   *
+   * @param scope the scope of the constraint
+   * @param minCardinality the minimum number of times the target expressions
+   *          have to be satisfied with the given scope
+   * @param maxCardinality the maximum number of times the target expressions
+   *          have to be satisfied with the given scope
+   * @param targetExpressions the target expressions
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint targetCardinality(String scope,
+      int minCardinality, int maxCardinality,
+      TargetExpression... targetExpressions) {
+    return new SingleConstraint(scope, minCardinality, maxCardinality,
+        targetExpressions);
+  }
+
+  // Creation of target expressions to be used in simple constraints.
+
+  /**
+   * Class with static methods for constructing target expressions to be used in
+   * placement constraints.
+   */
+  public static class PlacementTargets {
+
+    /**
+     * Constructs a target expression on a node attribute. It is satisfied if
+     * the specified node attribute has one of the specified values.
+     *
+     * @param attributeKey the name of the node attribute
+     * @param attributeValues the set of values that the attribute should take
+     *          values from
+     * @return the resulting expression on the node attribute
+     */
+    public static TargetExpression nodeAttribute(String attributeKey,
+        String... attributeValues) {
+      return new TargetExpression(TargetType.NODE_ATTRIBUTE, attributeKey,
+          attributeValues);
+    }
+
+    /**
+     * Constructs a target expression on an allocation tag. It is satisfied if
+     * the there are allocations with one of the given tags.
+     *
+     * @param allocationTags the set of tags that the attribute should take
+     *          values from
+     * @return the resulting expression on the allocation tags
+     */
+    public static TargetExpression allocationTag(String... allocationTags) {
+      return new TargetExpression(TargetType.ALLOCATION_TAG, null,
+          allocationTags);
+    }
+
+    /**
+     * The default target expression that uses as target the allocation that
+     * specifies the constraint.
+     *
+     * @return the self-target
+     */
+    public static TargetExpression self() {
+      return new TargetExpression(TargetType.SELF);
+    }
+  }
+
+  // Creation of compound constraints.
+
+  /**
+   * A conjunction of constraints.
+   *
+   * @param children the children constraints that should all be satisfied
+   * @return the resulting placement constraint
+   */
+  public static And and(AbstractConstraint... children) {
+    return new And(children);
+  }
+
+  /**
+   * A disjunction of constraints.
+   *
+   * @param children the children constraints, one of which should be satisfied
+   * @return the resulting placement constraint
+   */
+  public static Or or(AbstractConstraint... children) {
+    return new Or(children);
+  }
+
+  /**
+   * Creates a composite constraint that includes a list of timed placement
+   * constraints. The scheduler should try to satisfy first the first timed
+   * child constraint within the specified time window. If this is not possible,
+   * it should attempt to satisfy the second, and so on.
+   *
+   * @param children the timed children constraints
+   * @return the resulting composite constraint
+   */
+  public static DelayedOr delayedOr(TimedPlacementConstraint... children) {
+    return new DelayedOr(children);
+  }
+
+  // Creation of timed constraints to be used in a DELAYED_OR constraint.
+
+  /**
+   * Creates a placement constraint that has to be satisfied within a time
+   * window.
+   *
+   * @param constraint the placement constraint
+   * @param delay the length of the time window within which the constraint has
+   *          to be satisfied
+   * @param timeUnit the unit of time of the time window
+   * @return the resulting timed placement constraint
+   */
+  public static TimedPlacementConstraint timedClockConstraint(
+      AbstractConstraint constraint, long delay, TimeUnit timeUnit) {
+    return new TimedPlacementConstraint(constraint, timeUnit.toMillis(delay),
+        TimedPlacementConstraint.DelayUnit.MILLISECONDS);
+  }
+
+  /**
+   * Creates a placement constraint that has to be satisfied within a number of
+   * placement opportunities (invocations of the scheduler).
+   *
+   * @param constraint the placement constraint
+   * @param delay the number of scheduling opportunities within which the
+   *          constraint has to be satisfied
+   * @return the resulting timed placement constraint
+   */
+  public static TimedPlacementConstraint timedOpportunitiesConstraint(
+      AbstractConstraint constraint, long delay) {
+    return new TimedPlacementConstraint(constraint, delay,
+        TimedPlacementConstraint.DelayUnit.OPPORTUNITIES);
+  }
+
+  /**
+   * Creates a {@link PlacementConstraint} given a constraint expression.
+   *
+   * @param constraintExpr the constraint expression
+   * @return the placement constraint
+   */
+  public static PlacementConstraint build(AbstractConstraint constraintExpr) {
+    return constraintExpr.build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
new file mode 100644
index 0000000..660dc02
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * API related to resources.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.api.resource;
+import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index b6ea5f9..ff0d54b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -579,6 +579,61 @@ enum SignalContainerCommandProto {
   FORCEFUL_SHUTDOWN = 3;
 }
 
+////////////////////////////////////////////////////////////////////////
+////// Placement constraints ///////////////////////////////////////////
+////////////////////////////////////////////////////////////////////////
+
+message PlacementConstraintProto {
+  optional SimplePlacementConstraintProto simpleConstraint = 1;
+  optional CompositePlacementConstraintProto compositeConstraint = 2;
+}
+
+message SimplePlacementConstraintProto {
+  required string scope = 1;
+  repeated PlacementConstraintTargetProto targetExpressions = 2;
+  optional int32 minCardinality = 3;
+  optional int32 maxCardinality = 4;
+}
+
+message PlacementConstraintTargetProto {
+  enum TargetType {
+    NODE_ATTRIBUTE = 1;
+    ALLOCATION_TAG = 2;
+    SELF = 3;
+  }
+
+  required TargetType targetType = 1;
+  optional string targetKey = 2;
+  repeated string targetValues = 3;
+}
+
+message TimedPlacementConstraintProto {
+  enum DelayUnit {
+    MILLISECONDS = 1;
+    OPPORTUNITIES = 2;
+  }
+
+  required PlacementConstraintProto placementConstraint = 1;
+  required int64 schedulingDelay = 2;
+  optional DelayUnit delayUnit = 3 [ default = MILLISECONDS ];
+}
+
+message CompositePlacementConstraintProto {
+  enum CompositeType {
+    // All children constraints have to be satisfied.
+    AND = 1;
+    // One of the children constraints has to be satisfied.
+    OR = 2;
+    // Attempt to satisfy the first child constraint for delays[0] units (e.g.,
+    // millisec or heartbeats). If this fails, try to satisfy the second child
+    // constraint for delays[1] units and so on.
+    DELAYED_OR = 3;
+  }
+
+  required CompositeType compositeType = 1;
+  repeated PlacementConstraintProto childConstraints = 2;
+  repeated TimedPlacementConstraintProto timedChildConstraints = 3;
+}
 
 ////////////////////////////////////////////////////////////////////////
 ////// From reservation_protocol /////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java
new file mode 100644
index 0000000..e25d477
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraints.java
@@ -0,0 +1,106 @@
+/**
+ * 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.resource;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.RACK;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.and;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetNotIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.nodeAttribute;
+
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression.TargetType;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class for the various static methods in
+ * {@link org.apache.hadoop.yarn.api.resource.PlacementConstraints}.
+ */
+public class TestPlacementConstraints {
+
+  @Test
+  public void testNodeAffinityToTag() {
+    AbstractConstraint constraintExpr =
+        targetIn(NODE, allocationTag("hbase-m"));
+
+    SingleConstraint sConstraint = (SingleConstraint) constraintExpr;
+    Assert.assertEquals(NODE, sConstraint.getScope());
+    Assert.assertEquals(1, sConstraint.getMinCardinality());
+    Assert.assertEquals(Integer.MAX_VALUE, sConstraint.getMaxCardinality());
+
+    Assert.assertEquals(1, sConstraint.getTargetExpressions().size());
+    TargetExpression tExpr =
+        sConstraint.getTargetExpressions().iterator().next();
+    Assert.assertNull(tExpr.getTargetKey());
+    Assert.assertEquals(TargetType.ALLOCATION_TAG, tExpr.getTargetType());
+    Assert.assertEquals(1, tExpr.getTargetValues().size());
+    Assert.assertEquals("hbase-m", tExpr.getTargetValues().iterator().next());
+
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+    Assert.assertNotNull(constraint.getConstraintExpr());
+  }
+
+  @Test
+  public void testNodeAntiAffinityToAttribute() {
+    AbstractConstraint constraintExpr =
+        targetNotIn(NODE, nodeAttribute("java", "1.8"));
+
+    SingleConstraint sConstraint = (SingleConstraint) constraintExpr;
+    Assert.assertEquals(NODE, sConstraint.getScope());
+    Assert.assertEquals(0, sConstraint.getMinCardinality());
+    Assert.assertEquals(0, sConstraint.getMaxCardinality());
+
+    Assert.assertEquals(1, sConstraint.getTargetExpressions().size());
+    TargetExpression tExpr =
+        sConstraint.getTargetExpressions().iterator().next();
+    Assert.assertEquals("java", tExpr.getTargetKey());
+    Assert.assertEquals(TargetType.NODE_ATTRIBUTE, tExpr.getTargetType());
+    Assert.assertEquals(1, tExpr.getTargetValues().size());
+    Assert.assertEquals("1.8", tExpr.getTargetValues().iterator().next());
+  }
+
+  @Test
+  public void testAndConstraint() {
+    AbstractConstraint constraintExpr =
+        and(targetIn(RACK, allocationTag("spark")), maxCardinality(NODE, 3),
+            targetCardinality(RACK, 2, 10, allocationTag("zk")));
+
+    And andExpr = (And) constraintExpr;
+    Assert.assertEquals(3, andExpr.getChildren().size());
+    SingleConstraint sConstr = (SingleConstraint) andExpr.getChildren().get(0);
+    TargetExpression tExpr = sConstr.getTargetExpressions().iterator().next();
+    Assert.assertEquals("spark", tExpr.getTargetValues().iterator().next());
+
+    sConstr = (SingleConstraint) andExpr.getChildren().get(1);
+    Assert.assertEquals(0, sConstr.getMinCardinality());
+    Assert.assertEquals(3, sConstr.getMaxCardinality());
+
+    sConstr = (SingleConstraint) andExpr.getChildren().get(2);
+    Assert.assertEquals(2, sConstr.getMinCardinality());
+    Assert.assertEquals(10, sConstr.getMaxCardinality());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
new file mode 100644
index 0000000..926b6fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
@@ -0,0 +1,116 @@
+/**
+ * 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.pb;
+
+import static org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto.CompositeType.AND;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.TimedPlacementConstraintProto;
+
+/**
+ * {@code PlacementConstraintFromProtoConverter} generates an
+ * {@link PlacementConstraint.AbstractConstraint} given a
+ * {@link PlacementConstraintProto}.
+ */
+@Private
+public class PlacementConstraintFromProtoConverter {
+
+  private PlacementConstraintProto constraintProto;
+
+  public PlacementConstraintFromProtoConverter(
+      PlacementConstraintProto constraintProto) {
+    this.constraintProto = constraintProto;
+  }
+
+  public PlacementConstraint convert() {
+    return new PlacementConstraint(convert(constraintProto));
+  }
+
+  private AbstractConstraint convert(PlacementConstraintProto proto) {
+    return proto.hasSimpleConstraint() ? convert(proto.getSimpleConstraint())
+        : convert(proto.getCompositeConstraint());
+  }
+
+  private SingleConstraint convert(SimplePlacementConstraintProto proto) {
+    Set<TargetExpression> targets = new HashSet<>();
+    for (PlacementConstraintTargetProto tp : proto.getTargetExpressionsList()) {
+      targets.add(convert(tp));
+    }
+
+    return new SingleConstraint(proto.getScope(), proto.getMinCardinality(),
+        proto.getMaxCardinality(), targets);
+  }
+
+  private TargetExpression convert(PlacementConstraintTargetProto proto) {
+    return new TargetExpression(
+        ProtoUtils.convertFromProtoFormat(proto.getTargetType()),
+        proto.hasTargetKey() ? proto.getTargetKey() : null,
+        new HashSet<>(proto.getTargetValuesList()));
+  }
+
+  private AbstractConstraint convert(CompositePlacementConstraintProto proto) {
+    switch (proto.getCompositeType()) {
+    case AND:
+    case OR:
+      List<AbstractConstraint> children = new ArrayList<>();
+      for (PlacementConstraintProto cp : proto.getChildConstraintsList()) {
+        children.add(convert(cp));
+      }
+      return (proto.getCompositeType() == AND) ? new And(children)
+          : new Or(children);
+    case DELAYED_OR:
+      List<TimedPlacementConstraint> tChildren = new ArrayList<>();
+      for (TimedPlacementConstraintProto cp : proto
+          .getTimedChildConstraintsList()) {
+        tChildren.add(convert(cp));
+      }
+      return new DelayedOr(tChildren);
+    default:
+      throw new YarnRuntimeException(
+          "Encountered unexpected type of composite constraint.");
+    }
+  }
+
+  private TimedPlacementConstraint convert(
+      TimedPlacementConstraintProto proto) {
+    AbstractConstraint pConstraint = convert(proto.getPlacementConstraint());
+
+    return new TimedPlacementConstraint(pConstraint, proto.getSchedulingDelay(),
+        ProtoUtils.convertFromProtoFormat(proto.getDelayUnit()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
new file mode 100644
index 0000000..7816e18
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
@@ -0,0 +1,174 @@
+/**
+ * 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.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CardinalityConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CompositeConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto.CompositeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.TimedPlacementConstraintProto;
+
+import com.google.protobuf.GeneratedMessage;
+
+/**
+ * {@code PlacementConstraintToProtoConverter} generates a
+ * {@link PlacementConstraintProto} given a
+ * {@link PlacementConstraint.AbstractConstraint}.
+ */
+@Private
+public class PlacementConstraintToProtoConverter
+    implements PlacementConstraint.Visitor<GeneratedMessage> {
+
+  private PlacementConstraint placementConstraint;
+
+  public PlacementConstraintToProtoConverter(
+      PlacementConstraint placementConstraint) {
+    this.placementConstraint = placementConstraint;
+  }
+
+  public PlacementConstraintProto convert() {
+    return (PlacementConstraintProto) placementConstraint.getConstraintExpr()
+        .accept(this);
+  }
+
+  @Override
+  public GeneratedMessage visit(SingleConstraint constraint) {
+    SimplePlacementConstraintProto.Builder sb =
+        SimplePlacementConstraintProto.newBuilder();
+
+    if (constraint.getScope() != null) {
+      sb.setScope(constraint.getScope());
+    }
+    sb.setMinCardinality(constraint.getMinCardinality());
+    sb.setMaxCardinality(constraint.getMaxCardinality());
+    if (constraint.getTargetExpressions() != null) {
+      for (TargetExpression target : constraint.getTargetExpressions()) {
+        sb.addTargetExpressions(
+            (PlacementConstraintTargetProto) target.accept(this));
+      }
+
+    }
+    SimplePlacementConstraintProto sProto = sb.build();
+
+    // Wrap around PlacementConstraintProto object.
+    PlacementConstraintProto.Builder pb = PlacementConstraintProto.newBuilder();
+    pb.setSimpleConstraint(sProto);
+    return pb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(TargetExpression target) {
+    PlacementConstraintTargetProto.Builder tb =
+        PlacementConstraintTargetProto.newBuilder();
+
+    tb.setTargetType(ProtoUtils.convertToProtoFormat(target.getTargetType()));
+    if (target.getTargetKey() != null) {
+      tb.setTargetKey(target.getTargetKey());
+    }
+    if (target.getTargetValues() != null) {
+      tb.addAllTargetValues(target.getTargetValues());
+    }
+    return tb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(TargetConstraint constraint) {
+    throw new YarnRuntimeException("Unexpected TargetConstraint found.");
+  }
+
+  @Override
+  public GeneratedMessage visit(CardinalityConstraint constraint) {
+    throw new YarnRuntimeException("Unexpected CardinalityConstraint found.");
+  }
+
+  private GeneratedMessage visitAndOr(
+      CompositeConstraint<AbstractConstraint> composite, CompositeType type) {
+    CompositePlacementConstraintProto.Builder cb =
+        CompositePlacementConstraintProto.newBuilder();
+
+    cb.setCompositeType(type);
+
+    for (AbstractConstraint c : composite.getChildren()) {
+      cb.addChildConstraints((PlacementConstraintProto) c.accept(this));
+    }
+    CompositePlacementConstraintProto cProto = cb.build();
+
+    // Wrap around PlacementConstraintProto object.
+    PlacementConstraintProto.Builder pb = PlacementConstraintProto.newBuilder();
+    pb.setCompositeConstraint(cProto);
+    return pb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(And constraint) {
+    return visitAndOr(constraint, CompositeType.AND);
+  }
+
+  @Override
+  public GeneratedMessage visit(Or constraint) {
+    return visitAndOr(constraint, CompositeType.OR);
+  }
+
+  @Override
+  public GeneratedMessage visit(DelayedOr constraint) {
+    CompositePlacementConstraintProto.Builder cb =
+        CompositePlacementConstraintProto.newBuilder();
+
+    cb.setCompositeType(CompositeType.DELAYED_OR);
+
+    for (TimedPlacementConstraint c : constraint.getChildren()) {
+      cb.addTimedChildConstraints(
+          (TimedPlacementConstraintProto) c.accept(this));
+    }
+    CompositePlacementConstraintProto cProto = cb.build();
+
+    // Wrap around PlacementConstraintProto object.
+    PlacementConstraintProto.Builder pb = PlacementConstraintProto.newBuilder();
+    pb.setCompositeConstraint(cProto);
+    return pb.build();
+  }
+
+  @Override
+  public GeneratedMessage visit(TimedPlacementConstraint constraint) {
+    TimedPlacementConstraintProto.Builder tb =
+        TimedPlacementConstraintProto.newBuilder();
+
+    tb.setDelayUnit(ProtoUtils.convertToProtoFormat(constraint.getDelayUnit()));
+    tb.setSchedulingDelay(constraint.getSchedulingDelay());
+    tb.setPlacementConstraint(
+        (PlacementConstraintProto) constraint.getConstraint().accept(this));
+
+    return tb.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java
new file mode 100644
index 0000000..18da80f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * API related to protobuf objects that are not backed by PBImpl classes.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.api.pb;
+import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
index f3e665b..168d864 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
@@ -56,6 +56,8 @@ import org.apache.hadoop.yarn.api.records.UpdateContainerError;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
 import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnProtos.AMCommandProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAccessTypeProto;
@@ -70,10 +72,12 @@ import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceVisibilityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeStateProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.TimedPlacementConstraintProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerRetryPolicyProto;
@@ -507,6 +511,29 @@ public class ProtoUtils {
     }
     return ret;
   }
+
+  public static PlacementConstraintTargetProto.TargetType convertToProtoFormat(
+          TargetExpression.TargetType t) {
+    return PlacementConstraintTargetProto.TargetType.valueOf(t.name());
+  }
+
+  public static TargetExpression.TargetType convertFromProtoFormat(
+          PlacementConstraintTargetProto.TargetType t) {
+    return TargetExpression.TargetType.valueOf(t.name());
+  }
+
+  /*
+   * TimedPlacementConstraint.DelayUnit
+   */
+  public static TimedPlacementConstraintProto.DelayUnit convertToProtoFormat(
+          TimedPlacementConstraint.DelayUnit u) {
+    return TimedPlacementConstraintProto.DelayUnit.valueOf(u.name());
+  }
+
+  public static TimedPlacementConstraint.DelayUnit convertFromProtoFormat(
+          TimedPlacementConstraintProto.DelayUnit u) {
+    return TimedPlacementConstraint.DelayUnit.valueOf(u.name());
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java
new file mode 100644
index 0000000..e9eda6f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraintTransformations.java
@@ -0,0 +1,209 @@
+/**
+ * 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.resource;
+
+import java.util.ListIterator;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CardinalityConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CompositeConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint.TargetOperator;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/**
+ * This class contains inner classes that define transformation on a
+ * {@link PlacementConstraint} expression.
+ */
+@Private
+public class PlacementConstraintTransformations {
+
+  /**
+   * The default implementation of the {@link PlacementConstraint.Visitor} that
+   * does a traversal of the constraint tree, performing no action for the lead
+   * constraints.
+   */
+  public static class AbstractTransformer
+      implements PlacementConstraint.Visitor<AbstractConstraint> {
+
+    private PlacementConstraint placementConstraint;
+
+    public AbstractTransformer(PlacementConstraint placementConstraint) {
+      this.placementConstraint = placementConstraint;
+    }
+
+    /**
+     * This method performs the transformation of the
+     * {@link #placementConstraint}.
+     *
+     * @return the transformed placement constraint.
+     */
+    public PlacementConstraint transform() {
+      AbstractConstraint constraintExpr =
+          placementConstraint.getConstraintExpr();
+
+      // Visit the constraint tree to perform the transformation.
+      constraintExpr = constraintExpr.accept(this);
+
+      return new PlacementConstraint(constraintExpr);
+    }
+
+    @Override
+    public AbstractConstraint visit(SingleConstraint constraint) {
+      // Do nothing.
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(TargetExpression expression) {
+      // Do nothing.
+      return null;
+    }
+
+    @Override
+    public AbstractConstraint visit(TargetConstraint constraint) {
+      // Do nothing.
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(CardinalityConstraint constraint) {
+      // Do nothing.
+      return constraint;
+    }
+
+    private AbstractConstraint visitAndOr(
+        CompositeConstraint<AbstractConstraint> constraint) {
+      for (ListIterator<AbstractConstraint> iter =
+          constraint.getChildren().listIterator(); iter.hasNext();) {
+        AbstractConstraint child = iter.next();
+        child = child.accept(this);
+        iter.set(child);
+      }
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(And constraint) {
+      return visitAndOr(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(Or constraint) {
+      return visitAndOr(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(DelayedOr constraint) {
+      constraint.getChildren().forEach(
+          child -> child.setConstraint(child.getConstraint().accept(this)));
+      return constraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(TimedPlacementConstraint constraint) {
+      // Do nothing.
+      return null;
+    }
+  }
+
+  /**
+   * Visits a {@link PlacementConstraint} tree and substitutes each
+   * {@link TargetConstraint} and {@link CardinalityConstraint} with an
+   * equivalent {@link SingleConstraint}.
+   */
+  public static class SingleConstraintTransformer extends AbstractTransformer {
+
+    public SingleConstraintTransformer(PlacementConstraint constraint) {
+      super(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(TargetConstraint constraint) {
+      AbstractConstraint newConstraint;
+      if (constraint.getOp() == TargetOperator.IN) {
+        newConstraint = new SingleConstraint(constraint.getScope(), 1,
+            Integer.MAX_VALUE, constraint.getTargetExpressions());
+      } else if (constraint.getOp() == TargetOperator.NOT_IN) {
+        newConstraint = new SingleConstraint(constraint.getScope(), 0, 0,
+            constraint.getTargetExpressions());
+      } else {
+        throw new YarnRuntimeException(
+            "Encountered unexpected type of constraint target operator: "
+                + constraint.getOp());
+      }
+      return newConstraint;
+    }
+
+    @Override
+    public AbstractConstraint visit(CardinalityConstraint constraint) {
+      return new SingleConstraint(constraint.getScope(),
+          constraint.getMinCardinality(), constraint.getMaxCardinality(),
+          new TargetExpression(TargetExpression.TargetType.SELF));
+    }
+  }
+
+  /**
+   * Visits a {@link PlacementConstraint} tree and, whenever possible,
+   * substitutes each {@link SingleConstraint} with a {@link TargetConstraint}
+   * or a {@link CardinalityConstraint}. When such a substitution is not
+   * possible, we keep the original {@link SingleConstraint}.
+   */
+  public static class SpecializedConstraintTransformer
+      extends AbstractTransformer {
+
+    public SpecializedConstraintTransformer(PlacementConstraint constraint) {
+      super(constraint);
+    }
+
+    @Override
+    public AbstractConstraint visit(SingleConstraint constraint) {
+      AbstractConstraint transformedConstraint = constraint;
+      // Check if it is a cardinality constraint.
+      if (constraint.getTargetExpressions().size() == 1) {
+        TargetExpression targetExpr =
+            constraint.getTargetExpressions().iterator().next();
+        if (targetExpr.getTargetType() == TargetExpression.TargetType.SELF) {
+          transformedConstraint = new CardinalityConstraint(
+              constraint.getScope(), constraint.getMinCardinality(),
+              constraint.getMaxCardinality());
+        }
+      }
+      // Check if it is a target constraint.
+      if (constraint.getMinCardinality() == 1
+          && constraint.getMaxCardinality() == Integer.MAX_VALUE) {
+        transformedConstraint = new TargetConstraint(TargetOperator.IN,
+            constraint.getScope(), constraint.getTargetExpressions());
+      } else if (constraint.getMinCardinality() == 0
+          && constraint.getMaxCardinality() == 0) {
+        transformedConstraint = new TargetConstraint(TargetOperator.NOT_IN,
+            constraint.getScope(), constraint.getTargetExpressions());
+      }
+
+      return transformedConstraint;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
new file mode 100644
index 0000000..660dc02
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/resource/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * API related to resources.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.api.resource;
+import org.apache.hadoop.classification.InterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java
new file mode 100644
index 0000000..bd245e2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPlacementConstraintPBConversion.java
@@ -0,0 +1,195 @@
+/**
+ * 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;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.RACK;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.cardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.or;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+
+import java.util.Iterator;
+
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintFromProtoConverter;
+import org.apache.hadoop.yarn.api.pb.PlacementConstraintToProtoConverter;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto.CompositeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class for {@link PlacementConstraintToProtoConverter} and
+ * {@link PlacementConstraintFromProtoConverter}.
+ */
+public class TestPlacementConstraintPBConversion {
+
+  @Test
+  public void testTargetConstraintProtoConverter() {
+    AbstractConstraint sConstraintExpr =
+        targetIn(NODE, allocationTag("hbase-m"));
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Convert to proto.
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(sConstraint);
+    PlacementConstraintProto protoConstraint = toProtoConverter.convert();
+
+    Assert.assertTrue(protoConstraint.hasSimpleConstraint());
+    Assert.assertFalse(protoConstraint.hasCompositeConstraint());
+    SimplePlacementConstraintProto sProto =
+        protoConstraint.getSimpleConstraint();
+    Assert.assertEquals(single.getScope(), sProto.getScope());
+    Assert.assertEquals(single.getMinCardinality(), sProto.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(), sProto.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions().size(),
+        sProto.getTargetExpressionsList().size());
+
+    // Convert from proto.
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(protoConstraint);
+    PlacementConstraint newConstraint = fromProtoConverter.convert();
+
+    AbstractConstraint newConstraintExpr = newConstraint.getConstraintExpr();
+    Assert.assertTrue(newConstraintExpr instanceof SingleConstraint);
+    SingleConstraint newSingle = (SingleConstraint) newConstraintExpr;
+    Assert.assertEquals(single.getScope(), newSingle.getScope());
+    Assert.assertEquals(single.getMinCardinality(),
+        newSingle.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(),
+        newSingle.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions(),
+        newSingle.getTargetExpressions());
+  }
+
+  @Test
+  public void testCardinalityConstraintProtoConverter() {
+    AbstractConstraint sConstraintExpr = cardinality(RACK, 3, 10);
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Convert to proto.
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(sConstraint);
+    PlacementConstraintProto protoConstraint = toProtoConverter.convert();
+
+    compareSimpleConstraintToProto(single, protoConstraint);
+
+    // Convert from proto.
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(protoConstraint);
+    PlacementConstraint newConstraint = fromProtoConverter.convert();
+
+    AbstractConstraint newConstraintExpr = newConstraint.getConstraintExpr();
+    Assert.assertTrue(newConstraintExpr instanceof SingleConstraint);
+    SingleConstraint newSingle = (SingleConstraint) newConstraintExpr;
+    compareSimpleConstraints(single, newSingle);
+  }
+
+  @Test
+  public void testCompositeConstraintProtoConverter() {
+    AbstractConstraint constraintExpr =
+        or(targetIn(RACK, allocationTag("spark")), maxCardinality(NODE, 3),
+            targetCardinality(RACK, 2, 10, allocationTag("zk")));
+    Assert.assertTrue(constraintExpr instanceof Or);
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+    Or orExpr = (Or) constraintExpr;
+
+    // Convert to proto.
+    PlacementConstraintToProtoConverter toProtoConverter =
+        new PlacementConstraintToProtoConverter(constraint);
+    PlacementConstraintProto protoConstraint = toProtoConverter.convert();
+
+    Assert.assertFalse(protoConstraint.hasSimpleConstraint());
+    Assert.assertTrue(protoConstraint.hasCompositeConstraint());
+    CompositePlacementConstraintProto cProto =
+        protoConstraint.getCompositeConstraint();
+
+    Assert.assertEquals(CompositeType.OR, cProto.getCompositeType());
+    Assert.assertEquals(3, cProto.getChildConstraintsCount());
+    Assert.assertEquals(0, cProto.getTimedChildConstraintsCount());
+    Iterator<AbstractConstraint> orChildren = orExpr.getChildren().iterator();
+    Iterator<PlacementConstraintProto> orProtoChildren =
+        cProto.getChildConstraintsList().iterator();
+    while (orChildren.hasNext() && orProtoChildren.hasNext()) {
+      AbstractConstraint orChild = orChildren.next();
+      PlacementConstraintProto orProtoChild = orProtoChildren.next();
+      compareSimpleConstraintToProto((SingleConstraint) orChild, orProtoChild);
+    }
+
+    // Convert from proto.
+    PlacementConstraintFromProtoConverter fromProtoConverter =
+        new PlacementConstraintFromProtoConverter(protoConstraint);
+    PlacementConstraint newConstraint = fromProtoConverter.convert();
+
+    AbstractConstraint newConstraintExpr = newConstraint.getConstraintExpr();
+    Assert.assertTrue(newConstraintExpr instanceof Or);
+    Or newOrExpr = (Or) newConstraintExpr;
+    Assert.assertEquals(3, newOrExpr.getChildren().size());
+    orChildren = orExpr.getChildren().iterator();
+    Iterator<AbstractConstraint> newOrChildren =
+        newOrExpr.getChildren().iterator();
+    while (orChildren.hasNext() && newOrChildren.hasNext()) {
+      AbstractConstraint orChild = orChildren.next();
+      AbstractConstraint newOrChild = newOrChildren.next();
+      compareSimpleConstraints((SingleConstraint) orChild,
+          (SingleConstraint) newOrChild);
+    }
+  }
+
+  private void compareSimpleConstraintToProto(SingleConstraint constraint,
+      PlacementConstraintProto proto) {
+    Assert.assertTrue(proto.hasSimpleConstraint());
+    Assert.assertFalse(proto.hasCompositeConstraint());
+    SimplePlacementConstraintProto sProto = proto.getSimpleConstraint();
+    Assert.assertEquals(constraint.getScope(), sProto.getScope());
+    Assert.assertEquals(constraint.getMinCardinality(),
+        sProto.getMinCardinality());
+    Assert.assertEquals(constraint.getMaxCardinality(),
+        sProto.getMaxCardinality());
+    Assert.assertEquals(constraint.getTargetExpressions().size(),
+        sProto.getTargetExpressionsList().size());
+  }
+
+  private void compareSimpleConstraints(SingleConstraint single,
+      SingleConstraint newSingle) {
+    Assert.assertEquals(single.getScope(), newSingle.getScope());
+    Assert.assertEquals(single.getMinCardinality(),
+        newSingle.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(),
+        newSingle.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions(),
+        newSingle.getTargetExpressions());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e464cbf7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java
new file mode 100644
index 0000000..1763735
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java
@@ -0,0 +1,183 @@
+/**
+ * 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.resource;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.RACK;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.cardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.maxCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.or;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetCardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.CardinalityConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetConstraint.TargetOperator;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraintTransformations.SingleConstraintTransformer;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraintTransformations.SpecializedConstraintTransformer;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class for {@link PlacementConstraintTransformations}.
+ */
+public class TestPlacementConstraintTransformations {
+
+  @Test
+  public void testTargetConstraint() {
+    AbstractConstraint sConstraintExpr =
+        targetIn(NODE, allocationTag("hbase-m"));
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Transform from SimpleConstraint to specialized TargetConstraint
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(sConstraint);
+    PlacementConstraint tConstraint = specTransformer.transform();
+
+    AbstractConstraint tConstraintExpr = tConstraint.getConstraintExpr();
+    Assert.assertTrue(tConstraintExpr instanceof TargetConstraint);
+
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    TargetConstraint target = (TargetConstraint) tConstraintExpr;
+    Assert.assertEquals(single.getScope(), target.getScope());
+    Assert.assertEquals(TargetOperator.IN, target.getOp());
+    Assert.assertEquals(single.getTargetExpressions(),
+        target.getTargetExpressions());
+
+    // Transform from specialized TargetConstraint to SimpleConstraint
+    SingleConstraintTransformer singleTransformer =
+        new SingleConstraintTransformer(tConstraint);
+    sConstraint = singleTransformer.transform();
+
+    sConstraintExpr = sConstraint.getConstraintExpr();
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+
+    single = (SingleConstraint) sConstraintExpr;
+    Assert.assertEquals(target.getScope(), single.getScope());
+    Assert.assertEquals(1, single.getMinCardinality());
+    Assert.assertEquals(Integer.MAX_VALUE, single.getMaxCardinality());
+    Assert.assertEquals(single.getTargetExpressions(),
+        target.getTargetExpressions());
+  }
+
+  @Test
+  public void testCardinalityConstraint() {
+    AbstractConstraint sConstraintExpr = cardinality(RACK, 3, 10);
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+    PlacementConstraint sConstraint =
+        PlacementConstraints.build(sConstraintExpr);
+
+    // Transform from SimpleConstraint to specialized CardinalityConstraint
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(sConstraint);
+    PlacementConstraint cConstraint = specTransformer.transform();
+
+    AbstractConstraint cConstraintExpr = cConstraint.getConstraintExpr();
+    Assert.assertTrue(cConstraintExpr instanceof CardinalityConstraint);
+
+    SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    CardinalityConstraint cardinality = (CardinalityConstraint) cConstraintExpr;
+    Assert.assertEquals(single.getScope(), cardinality.getScope());
+    Assert.assertEquals(single.getMinCardinality(),
+        cardinality.getMinCardinality());
+    Assert.assertEquals(single.getMaxCardinality(),
+        cardinality.getMaxCardinality());
+
+    // Transform from specialized CardinalityConstraint to SimpleConstraint
+    SingleConstraintTransformer singleTransformer =
+        new SingleConstraintTransformer(cConstraint);
+    sConstraint = singleTransformer.transform();
+
+    sConstraintExpr = sConstraint.getConstraintExpr();
+    Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
+
+    single = (SingleConstraint) sConstraintExpr;
+    Assert.assertEquals(cardinality.getScope(), single.getScope());
+    Assert.assertEquals(cardinality.getMinCardinality(),
+        single.getMinCardinality());
+    Assert.assertEquals(cardinality.getMaxCardinality(),
+        single.getMaxCardinality());
+    Assert.assertEquals(new HashSet<>(Arrays.asList(PlacementTargets.self())),
+        single.getTargetExpressions());
+  }
+
+  @Test
+  public void testTargetCardinalityConstraint() {
+    AbstractConstraint constraintExpr =
+        targetCardinality(RACK, 3, 10, allocationTag("zk"));
+    Assert.assertTrue(constraintExpr instanceof SingleConstraint);
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+
+    // Apply transformation. Should be a no-op.
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(constraint);
+    PlacementConstraint newConstraint = specTransformer.transform();
+
+    // The constraint expression should be the same.
+    Assert.assertEquals(constraintExpr, newConstraint.getConstraintExpr());
+  }
+
+  @Test
+  public void testCompositeConstraint() {
+    AbstractConstraint constraintExpr =
+        or(targetIn(RACK, allocationTag("spark")), maxCardinality(NODE, 3),
+            targetCardinality(RACK, 2, 10, allocationTag("zk")));
+    Assert.assertTrue(constraintExpr instanceof Or);
+    PlacementConstraint constraint = PlacementConstraints.build(constraintExpr);
+    Or orExpr = (Or) constraintExpr;
+    for (AbstractConstraint child : orExpr.getChildren()) {
+      Assert.assertTrue(child instanceof SingleConstraint);
+    }
+
+    // Apply transformation. Should transform target and cardinality constraints
+    // included in the composite constraint to specialized ones.
+    SpecializedConstraintTransformer specTransformer =
+        new SpecializedConstraintTransformer(constraint);
+    PlacementConstraint specConstraint = specTransformer.transform();
+
+    Or specOrExpr = (Or) specConstraint.getConstraintExpr();
+    List<AbstractConstraint> specChildren = specOrExpr.getChildren();
+    Assert.assertEquals(3, specChildren.size());
+    Assert.assertTrue(specChildren.get(0) instanceof TargetConstraint);
+    Assert.assertTrue(specChildren.get(1) instanceof CardinalityConstraint);
+    Assert.assertTrue(specChildren.get(2) instanceof SingleConstraint);
+
+    // Transform from specialized TargetConstraint to SimpleConstraint
+    SingleConstraintTransformer singleTransformer =
+        new SingleConstraintTransformer(specConstraint);
+    PlacementConstraint simConstraint = singleTransformer.transform();
+    Assert.assertTrue(constraintExpr instanceof Or);
+    Or simOrExpr = (Or) specConstraint.getConstraintExpr();
+    for (AbstractConstraint child : simOrExpr.getChildren()) {
+      Assert.assertTrue(child instanceof SingleConstraint);
+    }
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org