001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.master.balancer; 019 020import java.lang.reflect.Constructor; 021import java.util.Collection; 022import java.util.Collections; 023import java.util.List; 024import java.util.Objects; 025import java.util.Set; 026import java.util.stream.Collectors; 027import org.apache.hadoop.conf.Configurable; 028import org.apache.hadoop.conf.Configuration; 029import org.apache.hadoop.hbase.master.RegionPlan; 030import org.apache.hadoop.hbase.master.balancer.replicas.ReplicaKeyCache; 031import org.apache.hadoop.hbase.util.ReflectionUtils; 032import org.apache.yetus.audience.InterfaceAudience; 033import org.slf4j.Logger; 034import org.slf4j.LoggerFactory; 035 036import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; 037 038/** 039 * Balancer conditionals supplement cost functions in the {@link StochasticLoadBalancer}. Cost 040 * functions are insufficient and difficult to work with when making discrete decisions; this is 041 * because they operate on a continuous scale, and each cost function's multiplier affects the 042 * relative importance of every other cost function. So it is difficult to meaningfully and clearly 043 * value many aspects of your region distribution via cost functions alone. Conditionals allow you 044 * to very clearly define discrete rules that your balancer would ideally follow. To clarify, a 045 * conditional violation will not block a region assignment because we would prefer to have uptime 046 * than have perfectly intentional balance. But conditionals allow you to, for example, define that 047 * a region's primary and secondary should not live on the same rack. Another example, conditionals 048 * make it easy to define that system tables will ideally be isolated on their own RegionServer 049 * (without needing to manage distinct RegionServer groups). 050 */ 051@InterfaceAudience.Private 052final class BalancerConditionals implements Configurable { 053 054 private static final Logger LOG = LoggerFactory.getLogger(BalancerConditionals.class); 055 056 public static final String DISTRIBUTE_REPLICAS_KEY = 057 "hbase.master.balancer.stochastic.conditionals.distributeReplicas"; 058 public static final boolean DISTRIBUTE_REPLICAS_DEFAULT = false; 059 060 public static final String ISOLATE_META_TABLE_KEY = 061 "hbase.master.balancer.stochastic.conditionals.isolateMetaTable"; 062 public static final boolean ISOLATE_META_TABLE_DEFAULT = false; 063 064 public static final String ADDITIONAL_CONDITIONALS_KEY = 065 "hbase.master.balancer.stochastic.additionalConditionals"; 066 067 private Set<Class<? extends RegionPlanConditional>> conditionalClasses = Collections.emptySet(); 068 private Set<RegionPlanConditional> conditionals = Collections.emptySet(); 069 private Configuration conf; 070 071 static BalancerConditionals create() { 072 return new BalancerConditionals(); 073 } 074 075 private BalancerConditionals() { 076 } 077 078 boolean shouldRunBalancer(BalancerClusterState cluster) { 079 return isConditionalBalancingEnabled() && conditionals.stream() 080 .map(RegionPlanConditional::getCandidateGenerators).flatMap(Collection::stream) 081 .map(generator -> generator.getWeight(cluster)).anyMatch(weight -> weight > 0); 082 } 083 084 Set<Class<? extends RegionPlanConditional>> getConditionalClasses() { 085 return Set.copyOf(conditionalClasses); 086 } 087 088 Collection<RegionPlanConditional> getConditionals() { 089 return conditionals; 090 } 091 092 boolean isReplicaDistributionEnabled() { 093 return conditionalClasses.stream() 094 .anyMatch(DistributeReplicasConditional.class::isAssignableFrom); 095 } 096 097 boolean isTableIsolationEnabled() { 098 return conditionalClasses.contains(MetaTableIsolationConditional.class); 099 } 100 101 boolean isServerHostingIsolatedTables(BalancerClusterState cluster, int serverIdx) { 102 return conditionals.stream().filter(TableIsolationConditional.class::isInstance) 103 .map(TableIsolationConditional.class::cast) 104 .anyMatch(conditional -> conditional.isServerHostingIsolatedTables(cluster, serverIdx)); 105 } 106 107 boolean isConditionalBalancingEnabled() { 108 return !conditionalClasses.isEmpty(); 109 } 110 111 void clearConditionalWeightCaches() { 112 conditionals.stream().map(RegionPlanConditional::getCandidateGenerators) 113 .flatMap(Collection::stream) 114 .forEach(RegionPlanConditionalCandidateGenerator::clearWeightCache); 115 } 116 117 void loadClusterState(BalancerClusterState cluster) { 118 conditionals = conditionalClasses.stream().map(clazz -> createConditional(clazz, cluster)) 119 .filter(Objects::nonNull).collect(Collectors.toSet()); 120 } 121 122 /** 123 * Indicates whether the action is good for our conditional compliance. 124 * @param cluster The cluster state 125 * @param action The proposed action 126 * @return -1 if conditionals improve, 0 if neutral, 1 if conditionals degrade 127 */ 128 int getViolationCountChange(BalancerClusterState cluster, BalanceAction action) { 129 // Cluster is in pre-move state, so figure out the proposed violations 130 boolean isViolatingPost = isViolating(cluster, action); 131 cluster.doAction(action); 132 133 // Cluster is in post-move state, so figure out the original violations 134 BalanceAction undoAction = action.undoAction(); 135 boolean isViolatingPre = isViolating(cluster, undoAction); 136 137 // Reset cluster 138 cluster.doAction(undoAction); 139 140 if (isViolatingPre && isViolatingPost) { 141 return 0; 142 } else if (!isViolatingPre && isViolatingPost) { 143 return 1; 144 } else { 145 return -1; 146 } 147 } 148 149 /** 150 * Check if the proposed action violates conditionals 151 * @param cluster The cluster state 152 * @param action The proposed action 153 */ 154 boolean isViolating(BalancerClusterState cluster, BalanceAction action) { 155 conditionals.forEach(conditional -> conditional.setClusterState(cluster)); 156 if (conditionals.isEmpty()) { 157 return false; 158 } 159 List<RegionPlan> regionPlans = action.toRegionPlans(cluster); 160 for (RegionPlan regionPlan : regionPlans) { 161 if (isViolating(regionPlan)) { 162 return true; 163 } 164 } 165 return false; 166 } 167 168 private boolean isViolating(RegionPlan regionPlan) { 169 for (RegionPlanConditional conditional : conditionals) { 170 if (conditional.isViolating(regionPlan)) { 171 return true; 172 } 173 } 174 return false; 175 } 176 177 private RegionPlanConditional createConditional(Class<? extends RegionPlanConditional> clazz, 178 BalancerClusterState cluster) { 179 if (cluster == null) { 180 cluster = new BalancerClusterState(Collections.emptyMap(), null, null, null, null); 181 } 182 try { 183 Constructor<? extends RegionPlanConditional> ctor = 184 clazz.getDeclaredConstructor(BalancerConditionals.class, BalancerClusterState.class); 185 return ReflectionUtils.instantiate(clazz.getName(), ctor, this, cluster); 186 } catch (NoSuchMethodException e) { 187 LOG.warn("Cannot find constructor with Configuration and " 188 + "BalancerClusterState parameters for class '{}': {}", clazz.getName(), e.getMessage()); 189 } 190 return null; 191 } 192 193 @Override 194 public void setConf(Configuration conf) { 195 this.conf = conf; 196 ImmutableSet.Builder<Class<? extends RegionPlanConditional>> conditionalClasses = 197 ImmutableSet.builder(); 198 199 boolean distributeReplicas = 200 conf.getBoolean(DISTRIBUTE_REPLICAS_KEY, DISTRIBUTE_REPLICAS_DEFAULT); 201 if (distributeReplicas) { 202 conditionalClasses.add(DistributeReplicasConditional.class); 203 } 204 205 boolean isolateMetaTable = conf.getBoolean(ISOLATE_META_TABLE_KEY, ISOLATE_META_TABLE_DEFAULT); 206 if (isolateMetaTable) { 207 conditionalClasses.add(MetaTableIsolationConditional.class); 208 } 209 210 Class<?>[] classes = conf.getClasses(ADDITIONAL_CONDITIONALS_KEY); 211 for (Class<?> clazz : classes) { 212 if (!RegionPlanConditional.class.isAssignableFrom(clazz)) { 213 LOG.warn("Class {} is not a RegionPlanConditional", clazz.getName()); 214 continue; 215 } 216 conditionalClasses.add(clazz.asSubclass(RegionPlanConditional.class)); 217 } 218 this.conditionalClasses = conditionalClasses.build(); 219 ReplicaKeyCache.getInstance().setConf(conf); 220 loadClusterState(null); 221 } 222 223 @Override 224 public Configuration getConf() { 225 return conf; 226 } 227}