|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.hadoop.hbase.master.balancer; |
| 19 | + |
| 20 | +import java.lang.reflect.Constructor; |
| 21 | +import java.util.Collection; |
| 22 | +import java.util.Collections; |
| 23 | +import java.util.List; |
| 24 | +import java.util.Objects; |
| 25 | +import java.util.Set; |
| 26 | +import java.util.stream.Collectors; |
| 27 | +import org.apache.hadoop.conf.Configurable; |
| 28 | +import org.apache.hadoop.conf.Configuration; |
| 29 | +import org.apache.hadoop.hbase.master.RegionPlan; |
| 30 | +import org.apache.hadoop.hbase.util.ReflectionUtils; |
| 31 | +import org.apache.yetus.audience.InterfaceAudience; |
| 32 | +import org.slf4j.Logger; |
| 33 | +import org.slf4j.LoggerFactory; |
| 34 | + |
| 35 | +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; |
| 36 | + |
| 37 | +/** |
| 38 | + * Balancer conditionals supplement cost functions in the {@link StochasticLoadBalancer}. Cost |
| 39 | + * functions are insufficient and difficult to work with when making discrete decisions; this is |
| 40 | + * because they operate on a continuous scale, and each cost function's multiplier affects the |
| 41 | + * relative importance of every other cost function. So it is difficult to meaningfully and clearly |
| 42 | + * value many aspects of your region distribution via cost functions alone. Conditionals allow you |
| 43 | + * to very clearly define discrete rules that your balancer would ideally follow. To clarify, a |
| 44 | + * conditional violation will not block a region assignment because we would prefer to have uptime |
| 45 | + * than have perfectly intentional balance. But conditionals allow you to, for example, define that |
| 46 | + * a region's primary and secondary should not live on the same rack. Another example, conditionals |
| 47 | + * make it easy to define that system tables will ideally be isolated on their own RegionServer |
| 48 | + * (without needing to manage distinct RegionServer groups). Use of conditionals may cause an |
| 49 | + * extremely unbalanced cluster to exceed its max balancer runtime. This is necessary because |
| 50 | + * conditional candidate generation is quite expensive, and cutting it off early could prevent us |
| 51 | + * from finding a solution. |
| 52 | + */ |
| 53 | +@InterfaceAudience.Private |
| 54 | +final class BalancerConditionals implements Configurable { |
| 55 | + |
| 56 | + private static final Logger LOG = LoggerFactory.getLogger(BalancerConditionals.class); |
| 57 | + |
| 58 | + static final BalancerConditionals INSTANCE = new BalancerConditionals(); |
| 59 | + public static final String DISTRIBUTE_REPLICAS_KEY = |
| 60 | + "hbase.master.balancer.stochastic.conditionals.distributeReplicas"; |
| 61 | + public static final boolean DISTRIBUTE_REPLICAS_DEFAULT = false; |
| 62 | + |
| 63 | + public static final String ADDITIONAL_CONDITIONALS_KEY = |
| 64 | + "hbase.master.balancer.stochastic.additionalConditionals"; |
| 65 | + |
| 66 | + private Set<Class<? extends RegionPlanConditional>> conditionalClasses = Collections.emptySet(); |
| 67 | + private Set<RegionPlanConditional> conditionals = Collections.emptySet(); |
| 68 | + private Configuration conf; |
| 69 | + |
| 70 | + private BalancerConditionals() { |
| 71 | + } |
| 72 | + |
| 73 | + boolean shouldRunBalancer(BalancerClusterState cluster) { |
| 74 | + return isConditionalBalancingEnabled() && conditionals.stream() |
| 75 | + .map(RegionPlanConditional::getCandidateGenerators).flatMap(Collection::stream) |
| 76 | + .map(generator -> generator.getWeight(cluster)).anyMatch(weight -> weight > 0); |
| 77 | + } |
| 78 | + |
| 79 | + Set<Class<? extends RegionPlanConditional>> getConditionalClasses() { |
| 80 | + return Set.copyOf(conditionalClasses); |
| 81 | + } |
| 82 | + |
| 83 | + Collection<RegionPlanConditional> getConditionals() { |
| 84 | + return conditionals; |
| 85 | + } |
| 86 | + |
| 87 | + boolean isReplicaDistributionEnabled() { |
| 88 | + return conditionalClasses.contains(DistributeReplicasConditional.class); |
| 89 | + } |
| 90 | + |
| 91 | + boolean shouldSkipSloppyServerEvaluation() { |
| 92 | + return isConditionalBalancingEnabled(); |
| 93 | + } |
| 94 | + |
| 95 | + boolean isConditionalBalancingEnabled() { |
| 96 | + return !conditionalClasses.isEmpty(); |
| 97 | + } |
| 98 | + |
| 99 | + void clearConditionalWeightCaches() { |
| 100 | + conditionals.stream().map(RegionPlanConditional::getCandidateGenerators) |
| 101 | + .flatMap(Collection::stream) |
| 102 | + .forEach(RegionPlanConditionalCandidateGenerator::clearWeightCache); |
| 103 | + } |
| 104 | + |
| 105 | + void loadClusterState(BalancerClusterState cluster) { |
| 106 | + conditionals = conditionalClasses.stream().map(clazz -> createConditional(clazz, conf, cluster)) |
| 107 | + .filter(Objects::nonNull).collect(Collectors.toSet()); |
| 108 | + } |
| 109 | + |
| 110 | + /** |
| 111 | + * Indicates whether the action is good for our conditional compliance. |
| 112 | + * @param cluster The cluster state |
| 113 | + * @param action The proposed action |
| 114 | + * @return -1 if conditionals improve, 0 if neutral, 1 if conditionals degrade |
| 115 | + */ |
| 116 | + int getViolationCountChange(BalancerClusterState cluster, BalanceAction action) { |
| 117 | + boolean isViolatingPre = isViolating(cluster, action.undoAction()); |
| 118 | + boolean isViolatingPost = isViolating(cluster, action); |
| 119 | + if (isViolatingPre && isViolatingPost) { |
| 120 | + return 0; |
| 121 | + } else if (!isViolatingPre && isViolatingPost) { |
| 122 | + return 1; |
| 123 | + } else { |
| 124 | + return -1; |
| 125 | + } |
| 126 | + } |
| 127 | + |
| 128 | + /** |
| 129 | + * Check if the proposed action violates conditionals |
| 130 | + * @param cluster The cluster state |
| 131 | + * @param action The proposed action |
| 132 | + */ |
| 133 | + boolean isViolating(BalancerClusterState cluster, BalanceAction action) { |
| 134 | + conditionals.forEach(conditional -> conditional.refreshClusterState(cluster)); |
| 135 | + if (conditionals.isEmpty()) { |
| 136 | + return false; |
| 137 | + } |
| 138 | + List<RegionPlan> regionPlans = cluster.convertActionToPlans(action); |
| 139 | + for (RegionPlan regionPlan : regionPlans) { |
| 140 | + if (isViolating(regionPlan)) { |
| 141 | + return true; |
| 142 | + } |
| 143 | + } |
| 144 | + return false; |
| 145 | + } |
| 146 | + |
| 147 | + private boolean isViolating(RegionPlan regionPlan) { |
| 148 | + for (RegionPlanConditional conditional : conditionals) { |
| 149 | + if (conditional.isViolating(regionPlan)) { |
| 150 | + return true; |
| 151 | + } |
| 152 | + } |
| 153 | + return false; |
| 154 | + } |
| 155 | + |
| 156 | + private RegionPlanConditional createConditional(Class<? extends RegionPlanConditional> clazz, |
| 157 | + Configuration conf, BalancerClusterState cluster) { |
| 158 | + if (conf == null) { |
| 159 | + conf = new Configuration(); |
| 160 | + } |
| 161 | + if (cluster == null) { |
| 162 | + cluster = new BalancerClusterState(Collections.emptyMap(), null, null, null, null); |
| 163 | + } |
| 164 | + try { |
| 165 | + Constructor<? extends RegionPlanConditional> ctor = |
| 166 | + clazz.getDeclaredConstructor(Configuration.class, BalancerClusterState.class); |
| 167 | + return ReflectionUtils.instantiate(clazz.getName(), ctor, conf, cluster); |
| 168 | + } catch (NoSuchMethodException e) { |
| 169 | + LOG.warn("Cannot find constructor with Configuration and " |
| 170 | + + "BalancerClusterState parameters for class '{}': {}", clazz.getName(), e.getMessage()); |
| 171 | + } |
| 172 | + return null; |
| 173 | + } |
| 174 | + |
| 175 | + @Override |
| 176 | + public void setConf(Configuration conf) { |
| 177 | + this.conf = conf; |
| 178 | + ImmutableSet.Builder<Class<? extends RegionPlanConditional>> conditionalClasses = |
| 179 | + ImmutableSet.builder(); |
| 180 | + |
| 181 | + boolean distributeReplicas = |
| 182 | + conf.getBoolean(DISTRIBUTE_REPLICAS_KEY, DISTRIBUTE_REPLICAS_DEFAULT); |
| 183 | + if (distributeReplicas) { |
| 184 | + conditionalClasses.add(DistributeReplicasConditional.class); |
| 185 | + } |
| 186 | + |
| 187 | + Class<?>[] classes = conf.getClasses(ADDITIONAL_CONDITIONALS_KEY); |
| 188 | + for (Class<?> clazz : classes) { |
| 189 | + if (!RegionPlanConditional.class.isAssignableFrom(clazz)) { |
| 190 | + LOG.warn("Class {} is not a RegionPlanConditional", clazz.getName()); |
| 191 | + continue; |
| 192 | + } |
| 193 | + conditionalClasses.add(clazz.asSubclass(RegionPlanConditional.class)); |
| 194 | + } |
| 195 | + this.conditionalClasses = conditionalClasses.build(); |
| 196 | + loadClusterState(null); |
| 197 | + } |
| 198 | + |
| 199 | + @Override |
| 200 | + public Configuration getConf() { |
| 201 | + return conf; |
| 202 | + } |
| 203 | +} |
0 commit comments