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


##########
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 {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BalancerConditionals.class);
+
+  static final BalancerConditionals INSTANCE = new BalancerConditionals();
+  public static final String ISOLATE_SYSTEM_TABLES_KEY =
+    "hbase.master.balancer.stochastic.conditionals.isolateSystemTables";
+  public static final boolean ISOLATE_SYSTEM_TABLES_DEFAULT = false;
+
+  public static final String ISOLATE_META_TABLE_KEY =
+    "hbase.master.balancer.stochastic.conditionals.isolateMetaTable";
+  public static final boolean ISOLATE_META_TABLE_DEFAULT = false;
+
+  public static final String DISTRIBUTE_REPLICAS_CONDITIONALS_KEY =
+    "hbase.master.balancer.stochastic.conditionals.distributeReplicas";
+  public static final boolean DISTRIBUTE_REPLICAS_CONDITIONALS_DEFAULT = false;
+
+  public static final String ADDITIONAL_CONDITIONALS_KEY =
+    "hbase.master.balancer.stochastic.additionalConditionals";
+
+  private Set<Class<? extends RegionPlanConditional>> conditionalClasses = 
Collections.emptySet();
+  private Set<RegionPlanConditional> conditionals = Collections.emptySet();
+  private Configuration conf;
+
+  private BalancerConditionals() {
+  }
+
+  boolean shouldRunBalancer(BalancerClusterState cluster) {
+    return isConditionalBalancingEnabled() && conditionals.stream()
+      
.map(RegionPlanConditional::getCandidateGenerator).flatMap(Optional::stream)
+      .map(generator -> generator.getWeight(cluster)).anyMatch(weight -> 
weight > 0);
+  }
+
+  Set<Class<? extends RegionPlanConditional>> getConditionalClasses() {
+    return Set.copyOf(conditionalClasses);
+  }
+
+  Collection<RegionPlanConditional> getConditionals() {
+    return conditionals;
+  }
+
+  boolean isMetaTableIsolationEnabled() {
+    return conditionalClasses.contains(MetaTableIsolationConditional.class);
+  }
+
+  boolean isSystemTableIsolationEnabled() {
+    return conditionalClasses.contains(SystemTableIsolationConditional.class);
+  }
+
+  boolean isReplicaDistributionEnabled() {
+    return conditionalClasses.contains(DistributeReplicasConditional.class);
+  }
+
+  boolean shouldSkipSloppyServerEvaluation() {
+    return isConditionalBalancingEnabled();
+  }
+
+  void clearConditionalWeightCaches() {
+    conditionals.stream().map(RegionPlanConditional::getCandidateGenerator)
+      
.flatMap(Optional::stream).forEach(RegionPlanConditionalCandidateGenerator::clearWeightCache);
+  }
+
+  void loadConf(Configuration conf) {

Review Comment:
   You can have BalancerConditionals implement Configurable or BaseConfigurable 
to do this in a more consistent way 



-- 
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