rmdmattingly commented on code in PR #6543:
URL: https://github.com/apache/hbase/pull/6543#discussion_r1898731798


##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerClusterState.java:
##########
@@ -705,7 +709,41 @@ enum LocalityType {
     RACK
   }
 
-  public void doAction(BalanceAction action) {
+  public List<RegionPlan> convertActionToPlans(BalanceAction action) {

Review Comment:
   RegionPlans are a more straightforward interface than BalanceActions, 
because you don't have to do all of this switch nonsense. So the new 
RegionPlanConditional interface isn't concerned with BalanceActions — it's just 
working with RegionInfo and RegionPlan objects, for example: 
   `isViolatingServer(RegionPlan regionPlan, Set<RegionInfo> 
destinationRegions)`
   
   All this to say, this was a nice method to introduce so that we could 
convert BalanceActions to RegionPlans as necessary for conditional evaluations, 
and without altering the current BalancerClusterState in-place via `doAction`



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CandidateGenerator.java:
##########
@@ -28,6 +28,8 @@
 @InterfaceAudience.Private
 abstract class CandidateGenerator {
 
+  static double MAX_WEIGHT = 100_000;

Review Comment:
   I thought it would make sense to have a universal ceiling for candidate 
generator weights, rather than them just being arbitrary doubles on any scale



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalanceAction.java:
##########
@@ -28,6 +28,8 @@ enum Type {
     ASSIGN_REGION,
     MOVE_REGION,
     SWAP_REGIONS,
+    ISOLATE_TABLE,
+    MOVE_BATCH,

Review Comment:
   Our conditional candidate generators can frequently see pretty far into the 
future if they've gone to the trouble of deriving one very opinionated move 
anyway. So this is a nice way to represent multiple regions moves triggered by 
one candidate generation iteration



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CacheAwareLoadBalancer.java:
##########
@@ -68,12 +68,13 @@ public synchronized void loadConf(Configuration 
configuration) {
   }
 
   @Override
-  protected List<CandidateGenerator> createCandidateGenerators() {
-    List<CandidateGenerator> candidateGenerators = new ArrayList<>(2);
-    candidateGenerators.add(GeneratorFunctionType.LOAD.ordinal(),
+  protected Map<Class<? extends CandidateGenerator>, CandidateGenerator>
+    createCandidateGenerators() {

Review Comment:
   The list/ordinal management in balancers was pretty ugly imo. Instead, this 
should be a mapping of class to generator obj



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionals.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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 java.lang.reflect.Constructor;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
+
+/**
+ * Balancer conditionals supplement cost functions in the {@link 
StochasticLoadBalancer}. Cost
+ * functions are insufficient and difficult to work with when making discrete 
decisions; this is
+ * because they operate on a continuous scale, and each cost function's 
multiplier affects the
+ * relative importance of every other cost function. So it is difficult to 
meaningfully and clearly
+ * value many aspects of your region distribution via cost functions alone. 
Conditionals allow you
+ * to very clearly define discrete rules that your balancer would ideally 
follow. To clarify, a
+ * conditional violation will not block a region assignment because we would 
prefer to have uptime
+ * than have perfectly intentional balance. But conditionals allow you to, for 
example, define that
+ * a region's primary and secondary should not live on the same rack. Another 
example, conditionals
+ * make it easy to define that system tables will ideally be isolated on their 
own RegionServer.
+ */
[email protected]
+public final class BalancerConditionals {

Review Comment:
   This class basically acts as a simplifying wrapper around our conditional 
implementations. Often we'll want to infer something, like "is this class 
instantiated?" or we'll want to do something against every conditional — like 
re-weight them, or validate a move against them. This class gives us an easy 
place to make these things easier



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java:
##########
@@ -157,15 +157,20 @@ public class StochasticLoadBalancer extends 
BaseLoadBalancer {
   private RegionReplicaHostCostFunction regionReplicaHostCostFunction;
   private RegionReplicaRackCostFunction regionReplicaRackCostFunction;
 
-  protected List<CandidateGenerator> candidateGenerators;
+  protected Map<Class<? extends CandidateGenerator>, CandidateGenerator> 
candidateGenerators;
+  private Map<Class<? extends CandidateGenerator>, Double> weightsOfGenerators;

Review Comment:
   Just to reiterate, I think maps were a better idea here



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerClusterState.java:
##########
@@ -899,6 +958,48 @@ int[] addRegion(int[] regions, int regionIndex) {
     return newRegions;
   }
 
+  int[] removeRegions(int[] regions, Set<Integer> regionIndicesToRemove) {
+    // Calculate the size of the new regions array
+    int newSize = regions.length - regionIndicesToRemove.size();
+    if (newSize < 0) {
+      throw new IllegalStateException(
+        "Region indices mismatch: more regions to remove than in the regions 
array");
+    }

Review Comment:
   These methods make it easier to add/remove many indices from the BCS. This 
is nice for the MOVE_BATCH balance action, that I've justified for candidate 
generation performance reasons in another comment here.
   
   Also, the nicer error messaging here is a good win imo. Previously you'd 
just hit ArrayIndexOutOfBoundExceptions, or worse — erroneous moves — when you 
fumbled the state management of your mutable BalancerClusterState. This should 
help anyone down the road if they're debugging a custom conditional 
implementation 



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java:
##########
@@ -372,29 +389,37 @@ boolean needsBalance(TableName tableName, 
BalancerClusterState cluster) {
       return true;
     }
 
-    if (sloppyRegionServerExist(cs)) {
+    if (!balancerConditionals.shouldSkipSloppyServerEvaluation() && 
sloppyRegionServerExist(cs)) {

Review Comment:
   If we should figure out how to have these more nicely coexist, then I'm open 
to that feedback. In my opinion, based on my experience with the balancer, this 
is a fine tradeoff



##########
hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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 static 
org.apache.hadoop.hbase.master.balancer.CandidateGeneratorTestUtil.isTableIsolated;
+import static 
org.apache.hadoop.hbase.master.balancer.CandidateGeneratorTestUtil.runBalancerToExhaustion;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category(MediumTests.class)
+public class TestLargeClusterBalancingMetaTableIsolation {

Review Comment:
   These tests are really solid imo. They make it easy to setup balancer 
scenarios with 10s of thousands of regions and thousands of servers, and ensure 
that the balancer can find its way out of the situation in a reasonable amount 
of time. These tests all pass reliably in 30s-3min on my local machine, often 
on the faster end, though it's a little dependent on luck — ie, how hairy are 
the edge cases that we randomly find ourselves in



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/balancer/TestReplicaDistributionBalancerConditional.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.balancer;
+
+import static 
org.apache.hadoop.hbase.balancer.BalancerConditionalsTestUtil.validateAssertionsWithRetries;
+
+import java.util.List;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.balancer.BalancerConditionals;
+import org.apache.hadoop.hbase.master.balancer.DistributeReplicasConditional;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category(LargeTests.class)
+public class TestReplicaDistributionBalancerConditional {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    
HBaseClassTestRule.forClass(TestReplicaDistributionBalancerConditional.class);
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TestReplicaDistributionBalancerConditional.class);
+  private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
+  private static final int REPLICAS = 3;
+  private static final int NUM_SERVERS = REPLICAS;
+  private static final int REGIONS_PER_SERVER = 5;
+
+  @Before
+  public void setUp() throws Exception {
+    TEST_UTIL.getConfiguration()
+      .setBoolean(BalancerConditionals.DISTRIBUTE_REPLICAS_CONDITIONALS_KEY, 
true);
+    
TEST_UTIL.getConfiguration().setBoolean(DistributeReplicasConditional.TEST_MODE_ENABLED_KEY,
+      true);
+    TEST_UTIL.getConfiguration()
+      .setBoolean(ServerRegionReplicaUtil.REGION_REPLICA_REPLICATION_CONF_KEY, 
true);
+    TEST_UTIL.getConfiguration().setLong(HConstants.HBASE_BALANCER_PERIOD, 
1000L);
+    
TEST_UTIL.getConfiguration().setBoolean("hbase.master.balancer.stochastic.runMaxSteps",
 true);
+
+    // turn off replica cost functions
+    TEST_UTIL.getConfiguration()
+      .setLong("hbase.master.balancer.stochastic.regionReplicaRackCostKey", 0);
+    TEST_UTIL.getConfiguration()
+      .setLong("hbase.master.balancer.stochastic.regionReplicaHostCostKey", 0);
+
+    TEST_UTIL.startMiniCluster(NUM_SERVERS);

Review Comment:
   In addition to the StochasticLoadBalancer tests in hbase-balancer that can 
test huge scales of conditional balancing, I wrote these mini cluster tests to 
smoke test that the balancer changes work in a "real" environment. This is also 
nice for testing the edge cases on smaller scales — eg, "I have 3 replicas and 
3 servers, please distribute them!"



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java:
##########
@@ -408,34 +433,53 @@ boolean needsBalance(TableName tableName, 
BalancerClusterState cluster) {
 
   @RestrictedApi(explanation = "Should only be called in tests", link = "",
       allowedOnPath = ".*(/src/test/.*|StochasticLoadBalancer).java")
-  BalanceAction nextAction(BalancerClusterState cluster) {
-    return getRandomGenerator().generate(cluster);
+  Pair<CandidateGenerator, BalanceAction> nextAction(BalancerClusterState 
cluster) {
+    CandidateGenerator generator = getRandomGenerator(cluster);
+    return Pair.newPair(generator, generator.generate(cluster));
   }
 
   /**
    * Select the candidate generator to use based on the cost of cost 
functions. The chance of
    * selecting a candidate generator is propotional to the share of cost of 
all cost functions among
    * all cost functions that benefit from it.
    */
-  protected CandidateGenerator getRandomGenerator() {
+  protected CandidateGenerator getRandomGenerator(BalancerClusterState 
cluster) {
     double sum = 0;
-    for (int i = 0; i < weightsOfGenerators.length; i++) {
-      sum += weightsOfGenerators[i];
-      weightsOfGenerators[i] = sum;
+    for (Class<? extends CandidateGenerator> clazz : 
candidateGenerators.keySet()) {
+      sum += weightsOfGenerators.getOrDefault(clazz, 0.0);
     }
     if (sum == 0) {
-      return candidateGenerators.get(0);
+      return candidateGenerators.values().stream().findAny().orElseThrow();
+    }
+
+    for (Class<? extends CandidateGenerator> clazz : 
candidateGenerators.keySet()) {
+      weightsOfGenerators.put(clazz,
+        Math.min(CandidateGenerator.MAX_WEIGHT, weightsOfGenerators.get(clazz) 
/ sum));
     }
-    for (int i = 0; i < weightsOfGenerators.length; i++) {
-      weightsOfGenerators[i] /= sum;
+
+    for (RegionPlanConditional conditional : 
balancerConditionals.getConditionals()) {
+      Optional<RegionPlanConditionalCandidateGenerator> generator =
+        conditional.getCandidateGenerator();
+      if (generator.isPresent() && generator.get().getWeight(cluster) > 0) {
+        return generator.get();
+      }
     }
+
     double rand = ThreadLocalRandom.current().nextDouble();
-    for (int i = 0; i < weightsOfGenerators.length; i++) {
-      if (rand <= weightsOfGenerators[i]) {
-        return candidateGenerators.get(i);
+    Class<? extends CandidateGenerator> greatestWeightClazz = null;
+    double greatestWeight = 0;
+    for (Class<? extends CandidateGenerator> clazz : 
candidateGenerators.keySet()) {
+      double generatorWeight = weightsOfGenerators.get(clazz);
+      if (generatorWeight > greatestWeight) {
+        greatestWeight = generatorWeight;
+        greatestWeightClazz = clazz;
+      }
+      if (rand <= generatorWeight) {
+        return candidateGenerators.get(clazz);
       }
     }
-    return candidateGenerators.get(candidateGenerators.size() - 1);

Review Comment:
   Previously, this method would often just return the last candidate generator 
in the list. This refactored implementation does a few things differently:
   1. It prioritizes conditional candidate generators over all others
   2. In the non-conditional case, it prioritizes the generator with the 
highest weight instead of the last one in the (no longer existent) list



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/SystemTableIsolationConditional.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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 java.util.Optional;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * If enabled, this class will help the balancer ensure that system tables 
live on their own
+ * RegionServer. System tables will share one RegionServer! This conditional 
can be used in tandem
+ * with {@link MetaTableIsolationConditional} to add a second RegionServer 
specifically for meta
+ * table hosting. Configure this via {@link 
BalancerConditionals#ISOLATE_SYSTEM_TABLES_KEY}
+ */
[email protected]
+class SystemTableIsolationConditional extends RegionPlanConditional {
+
+  public SystemTableIsolationConditional(Configuration conf, 
BalancerClusterState cluster) {
+    super(conf, cluster);
+    boolean isolateMeta = 
conf.getBoolean(BalancerConditionals.ISOLATE_META_TABLE_KEY, false);
+    
SystemTableIsolationCandidateGenerator.INSTANCE.setIsolateMeta(isolateMeta);
+  }
+
+  @Override
+  Optional<RegionPlanConditionalCandidateGenerator> getCandidateGenerator() {
+    return Optional.of(SystemTableIsolationCandidateGenerator.INSTANCE);
+  }
+
+  @Override
+  public boolean isViolatingServer(RegionPlan regionPlan, Set<RegionInfo> 
serverRegions) {
+    RegionInfo regionBeingMoved = regionPlan.getRegionInfo();
+    boolean shouldIsolateMovingRegion = isRegionToIsolate(regionBeingMoved);
+    for (RegionInfo destinationRegion : serverRegions) {
+      if 
(destinationRegion.getEncodedName().equals(regionBeingMoved.getEncodedName())) {
+        // Skip the region being moved
+        continue;
+      }
+      if (shouldIsolateMovingRegion && !isRegionToIsolate(destinationRegion)) {
+        // Ensure every destination region is also a region to isolate
+        return true;
+      } else if (!shouldIsolateMovingRegion && 
isRegionToIsolate(destinationRegion)) {
+        // Ensure no destination region is a region to isolate
+        return true;
+      }
+    }
+    return false;

Review Comment:
   I can dedupe a lot of this logic with the MetaTable conditional, will do 
sometime soon



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/TableIsolationCandidateGenerator.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
[email protected]
+public abstract class TableIsolationCandidateGenerator
+  extends RegionPlanConditionalCandidateGenerator {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TableIsolationCandidateGenerator.class);
+
+  abstract boolean shouldBeIsolated(RegionInfo regionInfo);
+
+  @Override
+  BalanceAction generate(BalancerClusterState cluster) {
+    return generateCandidate(cluster, false);
+  }
+
+  BalanceAction generateCandidate(BalancerClusterState cluster, boolean 
isWeighing) {
+    if (
+      !BalancerConditionals.INSTANCE.isSystemTableIsolationEnabled()
+        && !BalancerConditionals.INSTANCE.isMetaTableIsolationEnabled()
+    ) {
+      return BalanceAction.NULL_ACTION;
+    }
+
+    List<MoveRegionAction> moves = new ArrayList<>();
+    for (int serverIdx = 0; serverIdx < cluster.numServers; serverIdx++) {
+      boolean hasRegionsToIsolate = false;
+      Set<Integer> regionsToMove = new HashSet<>();
+
+      // Check all regions on the server
+      for (int regionIdx : cluster.regionsPerServer[serverIdx]) {
+        RegionInfo regionInfo = cluster.regions[regionIdx];
+        if (shouldBeIsolated(regionInfo)) {
+          hasRegionsToIsolate = true;
+        } else {
+          regionsToMove.add(regionIdx);
+        }
+      }
+
+      // Generate non-system regions to move, if applicable
+      if (hasRegionsToIsolate && !regionsToMove.isEmpty()) {
+        for (int regionToMove : regionsToMove) {
+          for (int i = 0; i < cluster.numServers; i++) {
+            int targetServer = pickOtherRandomServer(cluster, serverIdx);
+            MoveRegionAction possibleMove =
+              new MoveRegionAction(regionToMove, serverIdx, targetServer);
+            if (!BalancerConditionals.INSTANCE.isViolating(cluster, 
possibleMove)) {
+              if (isWeighing) {
+                return possibleMove;
+              }
+              moves.add(possibleMove);
+              break;
+            }
+          }
+        }
+      }
+
+      // todo should there be logic to consolidate isolated regions on as few 
servers as
+      // conditionals allow? This gets complicated with replicas, etc

Review Comment:
   I think we will want to do this in the v1 balancer conditionals impl, so I 
will do so shortly. But this current build is working very well, so I wanted to 
push



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java:
##########
@@ -521,21 +568,52 @@ protected List<RegionPlan> balanceTable(TableName 
tableName,
     // Perform a stochastic walk to see if we can get a good fit.
     long step;
 
+    boolean improvedConditionals = false;
+    Map<Class<? extends CandidateGenerator>, Long> generatorToStepCount = new 
HashMap<>();
+    Map<Class<? extends CandidateGenerator>, Long> 
generatorToApprovedActionCount = new HashMap<>();

Review Comment:
   These aren't really necessary, but make debugging easier because you can 
understand which candidate generators lead to the given balancer plan. If we 
had this logging, I think we would've realized the flaws in getRandomGenerator 
earlier (it being prone to just picking the last generator in the list)



##########
hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CandidateGenerator.java:
##########
@@ -28,6 +28,8 @@
 @InterfaceAudience.Private
 abstract class CandidateGenerator {
 
+  static double MAX_WEIGHT = 100_000;

Review Comment:
   Admittedly, we could rip this out now that I've made the decision to just 
let conditional candidate generators work first, if they want to



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to