Browse Source

YARN-5616. Clean up WeightAdjuster. (Yufei Gu via kasha)

Karthik Kambatla 8 years ago
parent
commit
c0e492e50f

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -186,7 +186,6 @@ public class FairScheduler extends
                                       // an app can be reserved on
 
   protected boolean sizeBasedWeight; // Give larger weights to larger jobs
-  protected WeightAdjuster weightAdjuster; // Can be null for no weight adjuster
   protected boolean continuousSchedulingEnabled; // Continuous Scheduling enabled or not
   protected int continuousSchedulingSleepMs; // Sleep time for each pass in continuous scheduling
   private Comparator<FSSchedulerNode> nodeAvailableResourceComparator =
@@ -563,10 +562,6 @@ public class FairScheduler extends
       weight = Math.log1p(app.getDemand().getMemorySize()) / Math.log(2);
     }
     weight *= app.getPriority().getPriority();
-    if (weightAdjuster != null) {
-      // Run weight through the user-supplied weightAdjuster
-      weight = weightAdjuster.adjustWeight(app, weight);
-    }
     ResourceWeights resourceWeights = app.getResourceWeights();
     resourceWeights.setWeight((float)weight);
     return resourceWeights;

+ 0 - 60
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/NewAppWeightBooster.java

@@ -1,60 +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.scheduler.fair;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-
-/**
- * A {@link WeightAdjuster} implementation that gives a weight boost to new jobs
- * for a certain amount of time -- by default, a 3x weight boost for 60 seconds.
- * This can be used to make shorter jobs finish faster, emulating Shortest Job
- * First scheduling while not starving long jobs.
- */
-@Private
-@Unstable
-public class NewAppWeightBooster extends Configured implements WeightAdjuster {
-  private static final float DEFAULT_FACTOR = 3;
-  private static final long DEFAULT_DURATION = 5 * 60 * 1000;
-
-  private float factor;
-  private long duration;
-
-  public void setConf(Configuration conf) {
-    if (conf != null) {
-      factor = conf.getFloat("mapred.newjobweightbooster.factor",
-          DEFAULT_FACTOR);
-      duration = conf.getLong("mapred.newjobweightbooster.duration",
-          DEFAULT_DURATION);
-    }
-    super.setConf(conf);
-  }
-
-  public double adjustWeight(FSAppAttempt app, double curWeight) {
-    long start = app.getStartTime();
-    long now = System.currentTimeMillis();
-    if (now - start < duration) {
-      return curWeight * factor;
-    } else {
-      return curWeight;
-    }
-  }
-}

+ 0 - 36
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/WeightAdjuster.java

@@ -1,36 +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.scheduler.fair;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configurable;
-
-/**
- * A pluggable object for altering the weights of apps in the fair scheduler,
- * which is used for example by {@link NewAppWeightBooster} to give higher
- * weight to new jobs so that short jobs finish faster.
- *
- * May implement {@link Configurable} to access configuration parameters.
- */
-@Private
-@Unstable
-public interface WeightAdjuster {
-  public double adjustWeight(FSAppAttempt app, double curWeight);
-}