You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wc...@apache.org on 2019/08/27 13:30:46 UTC

[hbase] branch branch-2 updated: HBASE-22618 added the possibility to load custom cost functions

This is an automated email from the ASF dual-hosted git repository.

wchevreuil pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new a18be19  HBASE-22618 added the possibility to load custom cost functions
a18be19 is described below

commit a18be1911e5706bca9d320fb14a0cf0ba4b7f7c5
Author: Pierre Zemb <co...@pierrezemb.fr>
AuthorDate: Sat Aug 24 16:46:03 2019 +0200

    HBASE-22618 added the possibility to load custom cost functions
    
    Signed-off-by: Wellington Chevreuil <wc...@apache.org>
---
 .../master/balancer/StochasticLoadBalancer.java    | 115 ++++++++++++++-------
 .../hbase/master/balancer/DummyCostFunction.java   |  31 ++++++
 .../balancer/TestStochasticLoadBalancer.java       |  12 +++
 3 files changed, 123 insertions(+), 35 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index fefe083..db96dcb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -27,7 +27,9 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Random;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics;
@@ -47,6 +49,7 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Locality
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.MoveRegionAction;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.SwapRegionsAction;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -82,6 +85,13 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
  *   <li>hbase.master.balancer.stochastic.storefileSizeCost</li>
  * </ul>
  *
+ * <p>You can also add custom Cost function by setting the the following configuration value:</p>
+ * <ul>
+ *     <li>hbase.master.balancer.stochastic.additionalCostFunctions</li>
+ * </ul>
+ *
+ * <p>All custom Cost Functions needs to extends {@link StochasticLoadBalancer.CostFunction}</p>
+ *
  * <p>In addition to the above configurations, the balancer can be tuned by the following
  * configuration values:</p>
  * <ul>
@@ -117,6 +127,8 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
   private static final String TABLE_FUNCTION_SEP = "_";
   protected static final String MIN_COST_NEED_BALANCE_KEY =
       "hbase.master.balancer.stochastic.minCostNeedBalance";
+  protected static final String COST_FUNCTIONS_COST_FUNCTIONS_KEY =
+          "hbase.master.balancer.stochastic.additionalCostFunctions";
 
   protected static final Random RANDOM = new Random(System.currentTimeMillis());
   private static final Logger LOG = LoggerFactory.getLogger(StochasticLoadBalancer.class);
@@ -133,7 +145,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
 
   private List<CandidateGenerator> candidateGenerators;
   private CostFromRegionLoadFunction[] regionLoadFunctions;
-  private CostFunction[] costFunctions; // FindBugs: Wants this protected; IS2_INCONSISTENT_SYNC
+  private List<CostFunction> costFunctions; // FindBugs: Wants this protected; IS2_INCONSISTENT_SYNC
 
   // to save and report costs to JMX
   private Double curOverallCost = 0d;
@@ -195,24 +207,57 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     };
     regionReplicaHostCostFunction = new RegionReplicaHostCostFunction(conf);
     regionReplicaRackCostFunction = new RegionReplicaRackCostFunction(conf);
-    costFunctions = new CostFunction[]{
-      new RegionCountSkewCostFunction(conf),
-      new PrimaryRegionCountSkewCostFunction(conf),
-      new MoveCostFunction(conf),
-      localityCost,
-      rackLocalityCost,
-      new TableSkewCostFunction(conf),
-      regionReplicaHostCostFunction,
-      regionReplicaRackCostFunction,
-      regionLoadFunctions[0],
-      regionLoadFunctions[1],
-      regionLoadFunctions[2],
-      regionLoadFunctions[3],
-    };
-    curFunctionCosts= new Double[costFunctions.length];
-    tempFunctionCosts= new Double[costFunctions.length];
+
+    costFunctions = new ArrayList<>();
+    costFunctions.add(new RegionCountSkewCostFunction(conf));
+    costFunctions.add(new PrimaryRegionCountSkewCostFunction(conf));
+    costFunctions.add(new MoveCostFunction(conf));
+    costFunctions.add(localityCost);
+    costFunctions.add(rackLocalityCost);
+    costFunctions.add(new TableSkewCostFunction(conf));
+    costFunctions.add(regionReplicaHostCostFunction);
+    costFunctions.add(regionReplicaRackCostFunction);
+    costFunctions.add(regionLoadFunctions[0]);
+    costFunctions.add(regionLoadFunctions[1]);
+    costFunctions.add(regionLoadFunctions[2]);
+    costFunctions.add(regionLoadFunctions[3]);
+    loadCustomCostFunctions(conf);
+
+    curFunctionCosts = new Double[costFunctions.size()];
+    tempFunctionCosts = new Double[costFunctions.size()];
+
     LOG.info("Loaded config; maxSteps=" + maxSteps + ", stepsPerRegion=" + stepsPerRegion +
-        ", maxRunningTime=" + maxRunningTime + ", isByTable=" + isByTable + ", etc.");
+            ", maxRunningTime=" + maxRunningTime + ", isByTable=" + isByTable + ", CostFunctions=" +
+            Arrays.toString(getCostFunctionNames()) + " etc.");
+  }
+
+  private void loadCustomCostFunctions(Configuration conf) {
+    String[] functionsNames = conf.getStrings(COST_FUNCTIONS_COST_FUNCTIONS_KEY);
+
+    if (null == functionsNames) {
+      return;
+    }
+
+    costFunctions.addAll(Arrays.stream(functionsNames)
+            .map(c -> {
+              Class<? extends CostFunction> klass = null;
+              try {
+                klass = (Class<? extends CostFunction>) Class.forName(c);
+              } catch (ClassNotFoundException e) {
+                LOG.warn("Cannot load class " + c + "': " + e.getMessage());
+              }
+              if (null == klass) {
+                return null;
+              }
+
+              CostFunction reflected = ReflectionUtils.newInstance(klass, conf);
+              LOG.info("Successfully loaded custom CostFunction '" +
+                      reflected.getClass().getSimpleName() + "'");
+
+              return reflected;
+            })
+            .filter(Objects::nonNull)
+            .collect(Collectors.toList()));
   }
 
   protected void setCandidateGenerators(List<CandidateGenerator> customCandidateGenerators) {
@@ -466,8 +511,8 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
         "Overall", "Overall cost", overall);
 
       // each cost function
-      for (int i = 0; i < costFunctions.length; i++) {
-        CostFunction costFunction = costFunctions[i];
+      for (int i = 0; i < costFunctions.size(); i++) {
+        CostFunction costFunction = costFunctions.get(i);
         String costFunctionName = costFunction.getClass().getSimpleName();
         Double costPercent = (overall == 0) ? 0 : (subCosts[i] / overall);
         // TODO: cost function may need a specific description
@@ -565,9 +610,9 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
    */
   public String[] getCostFunctionNames() {
     if (costFunctions == null) return null;
-    String[] ret = new String[costFunctions.length];
-    for (int i = 0; i < costFunctions.length; i++) {
-      CostFunction c = costFunctions[i];
+    String[] ret = new String[costFunctions.size()];
+    for (int i = 0; i < costFunctions.size(); i++) {
+      CostFunction c = costFunctions.get(i);
       ret[i] = c.getClass().getSimpleName();
     }
 
@@ -586,8 +631,8 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
   protected double computeCost(Cluster cluster, double previousCost) {
     double total = 0;
 
-    for (int i = 0; i < costFunctions.length; i++) {
-      CostFunction c = costFunctions[i];
+    for (int i = 0; i < costFunctions.size(); i++) {
+      CostFunction c = costFunctions.get(i);
       this.tempFunctionCosts[i] = 0.0;
 
       if (c.getMultiplier() <= 0) {
@@ -970,13 +1015,13 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
   /**
    * Base class of StochasticLoadBalancer's Cost Functions.
    */
-  abstract static class CostFunction {
+  public abstract static class CostFunction {
 
     private float multiplier = 0;
 
     protected Cluster cluster;
 
-    CostFunction(Configuration c) {
+    public CostFunction(Configuration c) {
     }
 
     boolean isNeeded() {
@@ -1025,7 +1070,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     protected void regionMoved(int region, int oldServer, int newServer) {
     }
 
-    abstract double cost();
+    protected abstract double cost();
 
     @SuppressWarnings("checkstyle:linelength")
     /**
@@ -1122,7 +1167,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     }
 
     @Override
-    double cost() {
+    protected double cost() {
       // Try and size the max number of Moves, but always be prepared to move some.
       int maxMoves = Math.max((int) (cluster.numRegions * maxMovesPercent),
           DEFAULT_MAX_MOVES);
@@ -1157,7 +1202,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     }
 
     @Override
-    double cost() {
+    protected double cost() {
       if (stats == null || stats.length != cluster.numServers) {
         stats = new double[cluster.numServers];
       }
@@ -1189,7 +1234,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     }
 
     @Override
-    double cost() {
+    protected double cost() {
       if (!cluster.hasRegionReplicas) {
         return 0;
       }
@@ -1226,7 +1271,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     }
 
     @Override
-    double cost() {
+    protected double cost() {
       double max = cluster.numRegions;
       double min = ((double) cluster.numRegions) / cluster.numServers;
       double value = 0;
@@ -1309,7 +1354,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     }
 
     @Override
-    double cost() {
+    protected double cost() {
       return 1 - locality;
     }
 
@@ -1387,7 +1432,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     }
 
     @Override
-    double cost() {
+    protected double cost() {
       if (clusterStatus == null || loads == null) {
         return 0;
       }
@@ -1557,7 +1602,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
     }
 
     @Override
-    double cost() {
+    protected double cost() {
       if (maxCost <= 0) {
         return 0;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/DummyCostFunction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/DummyCostFunction.java
new file mode 100644
index 0000000..680ffed
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/DummyCostFunction.java
@@ -0,0 +1,31 @@
+/**
+ * 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.hbase.master.balancer;
+
+import org.apache.hadoop.conf.Configuration;
+
+public class DummyCostFunction extends StochasticLoadBalancer.CostFunction {
+  public DummyCostFunction(Configuration c) {
+    super(c);
+  }
+
+  @Override
+  protected double cost() {
+    return 0;
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
index 41c3c4e..034e3e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java
@@ -25,6 +25,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
@@ -370,6 +371,17 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
     assertNull(plans);
   }
 
+  @Test
+  public void testAdditionalCostFunction() {
+    conf.set(StochasticLoadBalancer.COST_FUNCTIONS_COST_FUNCTIONS_KEY,
+            DummyCostFunction.class.getName());
+
+    loadBalancer.setConf(conf);
+    assertTrue(Arrays.
+            asList(loadBalancer.getCostFunctionNames()).
+            contains(DummyCostFunction.class.getSimpleName()));
+  }
+
   // This mock allows us to test the LocalityCostFunction
   private class MockCluster extends BaseLoadBalancer.Cluster {