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 ISOLATE_SYSTEM_TABLES_KEY =
065    "hbase.master.balancer.stochastic.conditionals.isolateSystemTables";
066  public static final boolean ISOLATE_SYSTEM_TABLES_DEFAULT = false;
067
068  public static final String ADDITIONAL_CONDITIONALS_KEY =
069    "hbase.master.balancer.stochastic.additionalConditionals";
070
071  private Set<Class<? extends RegionPlanConditional>> conditionalClasses = Collections.emptySet();
072  private Set<RegionPlanConditional> conditionals = Collections.emptySet();
073  private Configuration conf;
074
075  static BalancerConditionals create() {
076    return new BalancerConditionals();
077  }
078
079  private BalancerConditionals() {
080  }
081
082  boolean shouldRunBalancer(BalancerClusterState cluster) {
083    return isConditionalBalancingEnabled() && conditionals.stream()
084      .map(RegionPlanConditional::getCandidateGenerators).flatMap(Collection::stream)
085      .map(generator -> generator.getWeight(cluster)).anyMatch(weight -> weight > 0);
086  }
087
088  Set<Class<? extends RegionPlanConditional>> getConditionalClasses() {
089    return Set.copyOf(conditionalClasses);
090  }
091
092  Collection<RegionPlanConditional> getConditionals() {
093    return conditionals;
094  }
095
096  boolean isReplicaDistributionEnabled() {
097    return conditionalClasses.stream()
098      .anyMatch(DistributeReplicasConditional.class::isAssignableFrom);
099  }
100
101  boolean isTableIsolationEnabled() {
102    return conditionalClasses.stream().anyMatch(TableIsolationConditional.class::isAssignableFrom);
103  }
104
105  boolean isMetaTableIsolationEnabled() {
106    return conditionalClasses.contains(MetaTableIsolationConditional.class);
107  }
108
109  boolean isServerHostingIsolatedTables(BalancerClusterState cluster, int serverIdx) {
110    return conditionals.stream().filter(TableIsolationConditional.class::isInstance)
111      .map(TableIsolationConditional.class::cast)
112      .anyMatch(conditional -> conditional.isServerHostingIsolatedTables(cluster, serverIdx));
113  }
114
115  boolean isConditionalBalancingEnabled() {
116    return !conditionalClasses.isEmpty();
117  }
118
119  void clearConditionalWeightCaches() {
120    conditionals.stream().map(RegionPlanConditional::getCandidateGenerators)
121      .flatMap(Collection::stream)
122      .forEach(RegionPlanConditionalCandidateGenerator::clearWeightCache);
123  }
124
125  void loadClusterState(BalancerClusterState cluster) {
126    conditionals = conditionalClasses.stream().map(clazz -> createConditional(clazz, cluster))
127      .filter(Objects::nonNull).collect(Collectors.toSet());
128  }
129
130  /**
131   * Indicates whether the action is good for our conditional compliance.
132   * @param cluster The cluster state
133   * @param action  The proposed action
134   * @return -1 if conditionals improve, 0 if neutral, 1 if conditionals degrade
135   */
136  int getViolationCountChange(BalancerClusterState cluster, BalanceAction action) {
137    // Cluster is in pre-move state, so figure out the proposed violations
138    boolean isViolatingPost = isViolating(cluster, action);
139    cluster.doAction(action);
140
141    // Cluster is in post-move state, so figure out the original violations
142    BalanceAction undoAction = action.undoAction();
143    boolean isViolatingPre = isViolating(cluster, undoAction);
144
145    // Reset cluster
146    cluster.doAction(undoAction);
147
148    if (isViolatingPre && isViolatingPost) {
149      return 0;
150    } else if (!isViolatingPre && isViolatingPost) {
151      return 1;
152    } else {
153      return -1;
154    }
155  }
156
157  /**
158   * Check if the proposed action violates conditionals
159   * @param cluster The cluster state
160   * @param action  The proposed action
161   */
162  boolean isViolating(BalancerClusterState cluster, BalanceAction action) {
163    conditionals.forEach(conditional -> conditional.setClusterState(cluster));
164    if (conditionals.isEmpty()) {
165      return false;
166    }
167    List<RegionPlan> regionPlans = action.toRegionPlans(cluster);
168    for (RegionPlan regionPlan : regionPlans) {
169      if (isViolating(regionPlan)) {
170        return true;
171      }
172    }
173    return false;
174  }
175
176  private boolean isViolating(RegionPlan regionPlan) {
177    for (RegionPlanConditional conditional : conditionals) {
178      if (conditional.isViolating(regionPlan)) {
179        return true;
180      }
181    }
182    return false;
183  }
184
185  private RegionPlanConditional createConditional(Class<? extends RegionPlanConditional> clazz,
186    BalancerClusterState cluster) {
187    if (cluster == null) {
188      cluster = new BalancerClusterState(Collections.emptyMap(), null, null, null, null);
189    }
190    try {
191      Constructor<? extends RegionPlanConditional> ctor =
192        clazz.getDeclaredConstructor(BalancerConditionals.class, BalancerClusterState.class);
193      return ReflectionUtils.instantiate(clazz.getName(), ctor, this, cluster);
194    } catch (NoSuchMethodException e) {
195      LOG.warn("Cannot find constructor with Configuration and "
196        + "BalancerClusterState parameters for class '{}': {}", clazz.getName(), e.getMessage());
197    }
198    return null;
199  }
200
201  @Override
202  public void setConf(Configuration conf) {
203    this.conf = conf;
204    ImmutableSet.Builder<Class<? extends RegionPlanConditional>> conditionalClasses =
205      ImmutableSet.builder();
206
207    boolean distributeReplicas =
208      conf.getBoolean(DISTRIBUTE_REPLICAS_KEY, DISTRIBUTE_REPLICAS_DEFAULT);
209    if (distributeReplicas) {
210      conditionalClasses.add(DistributeReplicasConditional.class);
211    }
212
213    boolean isolateMetaTable = conf.getBoolean(ISOLATE_META_TABLE_KEY, ISOLATE_META_TABLE_DEFAULT);
214    if (isolateMetaTable) {
215      conditionalClasses.add(MetaTableIsolationConditional.class);
216    }
217
218    boolean isolateSystemTables =
219      conf.getBoolean(ISOLATE_SYSTEM_TABLES_KEY, ISOLATE_SYSTEM_TABLES_DEFAULT);
220    if (isolateSystemTables) {
221      conditionalClasses.add(SystemTableIsolationConditional.class);
222    }
223
224    Class<?>[] classes = conf.getClasses(ADDITIONAL_CONDITIONALS_KEY);
225    for (Class<?> clazz : classes) {
226      if (!RegionPlanConditional.class.isAssignableFrom(clazz)) {
227        LOG.warn("Class {} is not a RegionPlanConditional", clazz.getName());
228        continue;
229      }
230      conditionalClasses.add(clazz.asSubclass(RegionPlanConditional.class));
231    }
232    this.conditionalClasses = conditionalClasses.build();
233    ReplicaKeyCache.getInstance().setConf(conf);
234    loadClusterState(null);
235  }
236
237  @Override
238  public Configuration getConf() {
239    return conf;
240  }
241}