瀏覽代碼

Deleting files missed for YARN-827

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1495633 13f79535-47bb-0310-9956-ffa450edef68
Bikas Saha 12 年之前
父節點
當前提交
ba44df9166

+ 0 - 104
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DefaultResourceCalculator.java

@@ -1,104 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-package org.apache.hadoop.yarn.server.resourcemanager.resource;
-
-import org.apache.hadoop.yarn.api.records.Resource;
-
-public class DefaultResourceCalculator extends ResourceCalculator {
-
-  Resource clusterResource;
-  
-  @Override
-  public int compare(Resource unused, Resource lhs, Resource rhs) {
-    // Only consider memory
-    return lhs.getMemory() - rhs.getMemory();
-  }
-
-  @Override
-  public int computeAvailableContainers(Resource available, Resource required) {
-    // Only consider memory
-    return available.getMemory() / required.getMemory();
-  }
-
-  @Override
-  public float divide(Resource unused, 
-      Resource numerator, Resource denominator) {
-    return ratio(numerator, denominator);
-  }
-
-  @Override
-  public float ratio(Resource a, Resource b) {
-    return (float)a.getMemory() / b.getMemory();
-  }
-
-  @Override
-  public Resource divideAndCeil(Resource numerator, int denominator) {
-    return Resources.createResource(
-        divideAndCeil(numerator.getMemory(), denominator));
-  }
-
-  @Override
-  public Resource normalize(Resource r, Resource minimumResource,
-      Resource maximumResource, Resource stepFactor) {
-    int normalizedMemory = Math.min(
-        roundUp(
-            Math.max(r.getMemory(), minimumResource.getMemory()),
-            stepFactor.getMemory()),
-            maximumResource.getMemory());
-    return Resources.createResource(normalizedMemory);
-  }
-
-  @Override
-  public Resource normalize(Resource r, Resource minimumResource,
-                            Resource maximumResource) {
-    return normalize(r, minimumResource, maximumResource, minimumResource);
-  }
-
-  @Override
-  public Resource roundUp(Resource r, Resource stepFactor) {
-    return Resources.createResource(
-        roundUp(r.getMemory(), stepFactor.getMemory())
-        );
-  }
-
-  @Override
-  public Resource roundDown(Resource r, Resource stepFactor) {
-    return Resources.createResource(
-        roundDown(r.getMemory(), stepFactor.getMemory()));
-  }
-
-  @Override
-  public Resource multiplyAndNormalizeUp(Resource r, double by,
-      Resource stepFactor) {
-    return Resources.createResource(
-        roundUp((int)(r.getMemory() * by + 0.5), stepFactor.getMemory())
-        );
-  }
-
-  @Override
-  public Resource multiplyAndNormalizeDown(Resource r, double by,
-      Resource stepFactor) {
-    return Resources.createResource(
-        roundDown(
-            (int)(r.getMemory() * by), 
-            stepFactor.getMemory()
-            )
-        );
-  }
-
-}

+ 0 - 185
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DominantResourceCalculator.java

@@ -1,185 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-package org.apache.hadoop.yarn.server.resourcemanager.resource;
-
-import org.apache.hadoop.yarn.api.records.Resource;
-
-/**
- * A {@link ResourceCalculator} which uses the concept of  
- * <em>dominant resource</em> to compare multi-dimensional resources.
- *
- * Essentially the idea is that the in a multi-resource environment, 
- * the resource allocation should be determined by the dominant share 
- * of an entity (user or queue), which is the maximum share that the 
- * entity has been allocated of any resource. 
- * 
- * In a nutshell, it seeks to maximize the minimum dominant share across 
- * all entities. 
- * 
- * For example, if user A runs CPU-heavy tasks and user B runs
- * memory-heavy tasks, it attempts to equalize CPU share of user A 
- * with Memory-share of user B. 
- * 
- * In the single resource case, it reduces to max-min fairness for that resource.
- * 
- * See the Dominant Resource Fairness paper for more details:
- * www.cs.berkeley.edu/~matei/papers/2011/nsdi_drf.pdf
- */
-public class DominantResourceCalculator extends ResourceCalculator {
-  
-  @Override
-  public int compare(Resource clusterResource, Resource lhs, Resource rhs) {
-    
-    if (lhs.equals(rhs)) {
-      return 0;
-    }
-    
-    float l = getResourceAsValue(clusterResource, lhs, true);
-    float r = getResourceAsValue(clusterResource, rhs, true);
-    
-    if (l < r) {
-      return -1;
-    } else if (l > r) {
-      return 1;
-    } else {
-      l = getResourceAsValue(clusterResource, lhs, false);
-      r = getResourceAsValue(clusterResource, rhs, false);
-      if (l < r) {
-        return -1;
-      } else if (l > r) {
-        return 1;
-      }
-    }
-    
-    return 0;
-  }
-
-  /**
-   * Use 'dominant' for now since we only have 2 resources - gives us a slight
-   * performance boost.
-   * 
-   * Once we add more resources, we'll need a more complicated (and slightly
-   * less performant algorithm).
-   */
-  protected float getResourceAsValue(
-      Resource clusterResource, Resource resource, boolean dominant) {
-    // Just use 'dominant' resource
-    return (dominant) ?
-        Math.max(
-            (float)resource.getMemory() / clusterResource.getMemory(), 
-            (float)resource.getVirtualCores() / clusterResource.getVirtualCores()
-            ) 
-        :
-          Math.min(
-              (float)resource.getMemory() / clusterResource.getMemory(), 
-              (float)resource.getVirtualCores() / clusterResource.getVirtualCores()
-              ); 
-  }
-  
-  @Override
-  public int computeAvailableContainers(Resource available, Resource required) {
-    return Math.min(
-        available.getMemory() / required.getMemory(), 
-        available.getVirtualCores() / required.getVirtualCores());
-  }
-
-  @Override
-  public float divide(Resource clusterResource, 
-      Resource numerator, Resource denominator) {
-    return 
-        getResourceAsValue(clusterResource, numerator, true) / 
-        getResourceAsValue(clusterResource, denominator, true);
-  }
-
-  @Override
-  public float ratio(Resource a, Resource b) {
-    return Math.max(
-        (float)a.getMemory()/b.getMemory(), 
-        (float)a.getVirtualCores()/b.getVirtualCores()
-        );
-  }
-
-  @Override
-  public Resource divideAndCeil(Resource numerator, int denominator) {
-    return Resources.createResource(
-        divideAndCeil(numerator.getMemory(), denominator),
-        divideAndCeil(numerator.getVirtualCores(), denominator)
-        );
-  }
-
-  @Override
-  public Resource normalize(Resource r, Resource minimumResource,
-                            Resource maximumResource, Resource stepFactor) {
-    int normalizedMemory = Math.min(
-      roundUp(
-        Math.max(r.getMemory(), minimumResource.getMemory()),
-        stepFactor.getMemory()),
-      maximumResource.getMemory());
-    int normalizedCores = Math.min(
-      roundUp(
-        Math.max(r.getVirtualCores(), minimumResource.getVirtualCores()),
-        stepFactor.getVirtualCores()),
-      maximumResource.getVirtualCores());
-    return Resources.createResource(normalizedMemory,
-      normalizedCores);
-  }
-
-  @Override
-  public Resource roundUp(Resource r, Resource stepFactor) {
-    return Resources.createResource(
-        roundUp(r.getMemory(), stepFactor.getMemory()), 
-        roundUp(r.getVirtualCores(), stepFactor.getVirtualCores())
-        );
-  }
-
-  @Override
-  public Resource roundDown(Resource r, Resource stepFactor) {
-    return Resources.createResource(
-        roundDown(r.getMemory(), stepFactor.getMemory()),
-        roundDown(r.getVirtualCores(), stepFactor.getVirtualCores())
-        );
-  }
-
-  @Override
-  public Resource multiplyAndNormalizeUp(Resource r, double by,
-      Resource stepFactor) {
-    return Resources.createResource(
-        roundUp(
-            (int)Math.ceil(r.getMemory() * by), stepFactor.getMemory()),
-        roundUp(
-            (int)Math.ceil(r.getVirtualCores() * by), 
-            stepFactor.getVirtualCores())
-        );
-  }
-
-  @Override
-  public Resource multiplyAndNormalizeDown(Resource r, double by,
-      Resource stepFactor) {
-    return Resources.createResource(
-        roundDown(
-            (int)(r.getMemory() * by), 
-            stepFactor.getMemory()
-            ),
-        roundDown(
-            (int)(r.getVirtualCores() * by), 
-            stepFactor.getVirtualCores()
-            )
-        );
-  }
-
-}

+ 0 - 170
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceCalculator.java

@@ -1,170 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-package org.apache.hadoop.yarn.server.resourcemanager.resource;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.records.Resource;
-
-/**
- * A set of {@link Resource} comparison and manipulation interfaces.
- */
-@Private
-@Unstable
-public abstract class ResourceCalculator {
-
-  private static final Log LOG = LogFactory.getLog(ResourceCalculator.class);
-
-  public abstract int 
-  compare(Resource clusterResource, Resource lhs, Resource rhs);
-  
-  public static int divideAndCeil(int a, int b) {
-    if (b == 0) {
-      LOG.info("divideAndCeil called with a=" + a + " b=" + b);
-      return 0;
-    }
-    return (a + (b - 1)) / b;
-  }
-
-  public static int roundUp(int a, int b) {
-    return divideAndCeil(a, b) * b;
-  }
-
-  public static int roundDown(int a, int b) {
-    return (a / b) * b;
-  }
-
-  /**
-   * Compute the number of containers which can be allocated given
-   * <code>available</code> and <code>required</code> resources.
-   * 
-   * @param available available resources
-   * @param required required resources
-   * @return number of containers which can be allocated
-   */
-  public abstract int computeAvailableContainers(
-      Resource available, Resource required);
-
-  /**
-   * Multiply resource <code>r</code> by factor <code>by</code> 
-   * and normalize up using step-factor <code>stepFactor</code>.
-   * 
-   * @param r resource to be multiplied
-   * @param by multiplier
-   * @param stepFactor factor by which to normalize up 
-   * @return resulting normalized resource
-   */
-  public abstract Resource multiplyAndNormalizeUp(
-      Resource r, double by, Resource stepFactor);
-  
-  /**
-   * Multiply resource <code>r</code> by factor <code>by</code> 
-   * and normalize down using step-factor <code>stepFactor</code>.
-   * 
-   * @param r resource to be multiplied
-   * @param by multiplier
-   * @param stepFactor factor by which to normalize down 
-   * @return resulting normalized resource
-   */
-  public abstract Resource multiplyAndNormalizeDown(
-      Resource r, double by, Resource stepFactor);
-  
-  /**
-   * Normalize resource <code>r</code> given the base 
-   * <code>minimumResource</code> and verify against max allowed
-   * <code>maximumResource</code>
-   * 
-   * @param r resource
-   * @param minimumResource step-factor
-   * @param maximumResource the upper bound of the resource to be allocated
-   * @return normalized resource
-   */
-  public Resource normalize(Resource r, Resource minimumResource,
-      Resource maximumResource) {
-    return normalize(r, minimumResource, maximumResource, minimumResource);
-  }
-
-  /**
-   * Normalize resource <code>r</code> given the base 
-   * <code>minimumResource</code> and verify against max allowed
-   * <code>maximumResource</code> using a step factor for hte normalization.
-   *
-   * @param r resource
-   * @param minimumResource minimum value
-   * @param maximumResource the upper bound of the resource to be allocated
-   * @param stepFactor the increment for resources to be allocated
-   * @return normalized resource
-   */
-  public abstract Resource normalize(Resource r, Resource minimumResource,
-                                     Resource maximumResource, 
-                                     Resource stepFactor);
-
-
-  /**
-   * Round-up resource <code>r</code> given factor <code>stepFactor</code>.
-   * 
-   * @param r resource
-   * @param stepFactor step-factor
-   * @return rounded resource
-   */
-  public abstract Resource roundUp(Resource r, Resource stepFactor);
-  
-  /**
-   * Round-down resource <code>r</code> given factor <code>stepFactor</code>.
-   * 
-   * @param r resource
-   * @param stepFactor step-factor
-   * @return rounded resource
-   */
-  public abstract Resource roundDown(Resource r, Resource stepFactor);
-  
-  /**
-   * Divide resource <code>numerator</code> by resource <code>denominator</code>
-   * using specified policy (domination, average, fairness etc.); hence overall
-   * <code>clusterResource</code> is provided for context.
-   *  
-   * @param clusterResource cluster resources
-   * @param numerator numerator
-   * @param denominator denominator
-   * @return <code>numerator</code>/<code>denominator</code> 
-   *         using specific policy
-   */
-  public abstract float divide(
-      Resource clusterResource, Resource numerator, Resource denominator);
-  
-  /**
-   * Ratio of resource <code>a</code> to resource <code>b</code>.
-   * 
-   * @param a resource 
-   * @param b resource
-   * @return ratio of resource <code>a</code> to resource <code>b</code>
-   */
-  public abstract float ratio(Resource a, Resource b);
-
-  /**
-   * Divide-and-ceil <code>numerator</code> by <code>denominator</code>.
-   * 
-   * @param numerator numerator resource
-   * @param denominator denominator
-   * @return resultant resource
-   */
-  public abstract Resource divideAndCeil(Resource numerator, int denominator);
-  
-}

+ 0 - 221
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java

@@ -1,221 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.hadoop.yarn.server.resourcemanager.resource;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.util.Records;
-
-@Private
-@Evolving
-public class Resources {
-  
-  // Java doesn't have const :(
-  private static final Resource NONE = new Resource() {
-
-    @Override
-    public int getMemory() {
-      return 0;
-    }
-
-    @Override
-    public void setMemory(int memory) {
-      throw new RuntimeException("NONE cannot be modified!");
-    }
-
-    @Override
-    public int getVirtualCores() {
-      return 0;
-    }
-
-    @Override
-    public void setVirtualCores(int cores) {
-      throw new RuntimeException("NONE cannot be modified!");
-    }
-
-    @Override
-    public int compareTo(Resource o) {
-      int diff = 0 - o.getMemory();
-      if (diff == 0) {
-        diff = 0 - o.getVirtualCores();
-      }
-      return diff;
-    }
-    
-  };
-
-  public static Resource createResource(int memory) {
-    return createResource(memory, (memory > 0) ? 1 : 0);
-  }
-
-  public static Resource createResource(int memory, int cores) {
-    Resource resource = Records.newRecord(Resource.class);
-    resource.setMemory(memory);
-    resource.setVirtualCores(cores);
-    return resource;
-  }
-
-  public static Resource none() {
-    return NONE;
-  }
-
-  public static Resource clone(Resource res) {
-    return createResource(res.getMemory(), res.getVirtualCores());
-  }
-
-  public static Resource addTo(Resource lhs, Resource rhs) {
-    lhs.setMemory(lhs.getMemory() + rhs.getMemory());
-    lhs.setVirtualCores(lhs.getVirtualCores() + rhs.getVirtualCores());
-    return lhs;
-  }
-
-  public static Resource add(Resource lhs, Resource rhs) {
-    return addTo(clone(lhs), rhs);
-  }
-
-  public static Resource subtractFrom(Resource lhs, Resource rhs) {
-    lhs.setMemory(lhs.getMemory() - rhs.getMemory());
-    lhs.setVirtualCores(lhs.getVirtualCores() - rhs.getVirtualCores());
-    return lhs;
-  }
-
-  public static Resource subtract(Resource lhs, Resource rhs) {
-    return subtractFrom(clone(lhs), rhs);
-  }
-
-  public static Resource negate(Resource resource) {
-    return subtract(NONE, resource);
-  }
-
-  public static Resource multiplyTo(Resource lhs, double by) {
-    lhs.setMemory((int)(lhs.getMemory() * by));
-    lhs.setVirtualCores((int)(lhs.getVirtualCores() * by));
-    return lhs;
-  }
-
-  public static Resource multiply(Resource lhs, double by) {
-    return multiplyTo(clone(lhs), by);
-  }
-  
-  public static Resource multiplyAndNormalizeUp(
-      ResourceCalculator calculator,Resource lhs, double by, Resource factor) {
-    return calculator.multiplyAndNormalizeUp(lhs, by, factor);
-  }
-  
-  public static Resource multiplyAndNormalizeDown(
-      ResourceCalculator calculator,Resource lhs, double by, Resource factor) {
-    return calculator.multiplyAndNormalizeDown(lhs, by, factor);
-  }
-  
-  public static Resource multiplyAndRoundDown(Resource lhs, double by) {
-    Resource out = clone(lhs);
-    out.setMemory((int)(lhs.getMemory() * by));
-    out.setVirtualCores((int)(lhs.getVirtualCores() * by));
-    return out;
-  }
-  
-  public static Resource normalize(
-      ResourceCalculator calculator, Resource lhs, Resource min,
-      Resource max, Resource increment) {
-    return calculator.normalize(lhs, min, max, increment);
-  }
-  
-  public static Resource roundUp(
-      ResourceCalculator calculator, Resource lhs, Resource factor) {
-    return calculator.roundUp(lhs, factor);
-  }
-  
-  public static Resource roundDown(
-      ResourceCalculator calculator, Resource lhs, Resource factor) {
-    return calculator.roundDown(lhs, factor);
-  }
-  
-  public static float ratio(
-      ResourceCalculator resourceCalculator, Resource lhs, Resource rhs) {
-    return resourceCalculator.ratio(lhs, rhs);
-  }
-  
-  public static float divide(
-      ResourceCalculator resourceCalculator,
-      Resource clusterResource, Resource lhs, Resource rhs) {
-    return resourceCalculator.divide(clusterResource, lhs, rhs);
-  }
-  
-  public static Resource divideAndCeil(
-      ResourceCalculator resourceCalculator, Resource lhs, int rhs) {
-    return resourceCalculator.divideAndCeil(lhs, rhs);
-  }
-  
-  public static boolean equals(Resource lhs, Resource rhs) {
-    return lhs.equals(rhs);
-  }
-
-  public static boolean lessThan(
-      ResourceCalculator resourceCalculator, 
-      Resource clusterResource,
-      Resource lhs, Resource rhs) {
-    return (resourceCalculator.compare(clusterResource, lhs, rhs) < 0);
-  }
-
-  public static boolean lessThanOrEqual(
-      ResourceCalculator resourceCalculator, 
-      Resource clusterResource,
-      Resource lhs, Resource rhs) {
-    return (resourceCalculator.compare(clusterResource, lhs, rhs) <= 0);
-  }
-
-  public static boolean greaterThan(
-      ResourceCalculator resourceCalculator,
-      Resource clusterResource,
-      Resource lhs, Resource rhs) {
-    return resourceCalculator.compare(clusterResource, lhs, rhs) > 0;
-  }
-
-  public static boolean greaterThanOrEqual(
-      ResourceCalculator resourceCalculator, 
-      Resource clusterResource,
-      Resource lhs, Resource rhs) {
-    return resourceCalculator.compare(clusterResource, lhs, rhs) >= 0;
-  }
-  
-  public static Resource min(
-      ResourceCalculator resourceCalculator, 
-      Resource clusterResource,
-      Resource lhs, Resource rhs) {
-    return resourceCalculator.compare(clusterResource, lhs, rhs) <= 0 ? lhs : rhs;
-  }
-
-  public static Resource max(
-      ResourceCalculator resourceCalculator, 
-      Resource clusterResource,
-      Resource lhs, Resource rhs) {
-    return resourceCalculator.compare(clusterResource, lhs, rhs) >= 0 ? lhs : rhs;
-  }
-  
-  public static boolean fitsIn(Resource smaller, Resource bigger) {
-    return smaller.getMemory() <= bigger.getMemory() &&
-        smaller.getVirtualCores() <= bigger.getVirtualCores();
-  }
-  
-  public static Resource componentwiseMin(Resource lhs, Resource rhs) {
-    return createResource(Math.min(lhs.getMemory(), rhs.getMemory()),
-        Math.min(lhs.getVirtualCores(), rhs.getVirtualCores()));
-  }
-}