diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalanceAction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalanceAction.java index 56b473ae710c..ab9395fef163 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalanceAction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalanceAction.java @@ -28,6 +28,7 @@ enum Type { ASSIGN_REGION, MOVE_REGION, SWAP_REGIONS, + MOVE_BATCH, NULL, } @@ -51,6 +52,10 @@ Type getType() { return type; } + long getStepCount() { + return 1; + } + @Override public String toString() { return type + ":"; diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerClusterState.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerClusterState.java index 4b3809c107cb..4361f82ac676 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerClusterState.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerClusterState.java @@ -26,6 +26,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; import org.agrona.collections.Hashing; import org.agrona.collections.Int2IntCounterMap; import org.apache.hadoop.hbase.HDFSBlocksDistribution; @@ -33,12 +35,17 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.master.RackManager; +import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.net.Address; import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.base.Supplier; +import org.apache.hbase.thirdparty.com.google.common.base.Suppliers; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; + /** * An efficient array based implementation similar to ClusterState for keeping the status of the * cluster in terms of region assignment and distribution. LoadBalancers, such as @@ -122,6 +129,14 @@ class BalancerClusterState { // Maps regionName -> oldServerName -> cache ratio of the region on the old server Map> regionCacheRatioOnOldServerMap; + private Supplier> shuffledServerIndicesSupplier = + Suppliers.memoizeWithExpiration(() -> { + Collection serverIndices = serversToIndex.values(); + List shuffledServerIndices = new ArrayList<>(serverIndices); + Collections.shuffle(shuffledServerIndices); + return shuffledServerIndices; + }, 5, TimeUnit.SECONDS); + static class DefaultRackManager extends RackManager { @Override public String getRack(ServerName server) { @@ -705,7 +720,41 @@ enum LocalityType { RACK } - public void doAction(BalanceAction action) { + public List convertActionToPlans(BalanceAction action) { + switch (action.getType()) { + case NULL: + break; + case ASSIGN_REGION: + // FindBugs: Having the assert quietens FB BC_UNCONFIRMED_CAST warnings + assert action instanceof AssignRegionAction : action.getClass(); + AssignRegionAction ar = (AssignRegionAction) action; + return ImmutableList.of(regionMoved(ar.getRegion(), -1, ar.getServer())); + case MOVE_REGION: + assert action instanceof MoveRegionAction : action.getClass(); + MoveRegionAction mra = (MoveRegionAction) action; + return ImmutableList + .of(regionMoved(mra.getRegion(), mra.getFromServer(), mra.getToServer())); + case SWAP_REGIONS: + assert action instanceof SwapRegionsAction : action.getClass(); + SwapRegionsAction a = (SwapRegionsAction) action; + return ImmutableList.of(regionMoved(a.getFromRegion(), a.getFromServer(), a.getToServer()), + regionMoved(a.getToRegion(), a.getToServer(), a.getFromServer())); + case MOVE_BATCH: + assert action instanceof MoveBatchAction : action.getClass(); + MoveBatchAction mba = (MoveBatchAction) action; + List mbRegionPlans = new ArrayList<>(); + for (MoveRegionAction moveRegionAction : mba.getMoveActions()) { + mbRegionPlans.add(regionMoved(moveRegionAction.getRegion(), + moveRegionAction.getFromServer(), moveRegionAction.getToServer())); + } + return mbRegionPlans; + default: + throw new RuntimeException("Unknown action:" + action.getType()); + } + return Collections.emptyList(); + } + + public List doAction(BalanceAction action) { switch (action.getType()) { case NULL: break; @@ -715,8 +764,7 @@ public void doAction(BalanceAction action) { AssignRegionAction ar = (AssignRegionAction) action; regionsPerServer[ar.getServer()] = addRegion(regionsPerServer[ar.getServer()], ar.getRegion()); - regionMoved(ar.getRegion(), -1, ar.getServer()); - break; + return ImmutableList.of(regionMoved(ar.getRegion(), -1, ar.getServer())); case MOVE_REGION: assert action instanceof MoveRegionAction : action.getClass(); MoveRegionAction mra = (MoveRegionAction) action; @@ -724,8 +772,8 @@ public void doAction(BalanceAction action) { removeRegion(regionsPerServer[mra.getFromServer()], mra.getRegion()); regionsPerServer[mra.getToServer()] = addRegion(regionsPerServer[mra.getToServer()], mra.getRegion()); - regionMoved(mra.getRegion(), mra.getFromServer(), mra.getToServer()); - break; + return ImmutableList + .of(regionMoved(mra.getRegion(), mra.getFromServer(), mra.getToServer())); case SWAP_REGIONS: assert action instanceof SwapRegionsAction : action.getClass(); SwapRegionsAction a = (SwapRegionsAction) action; @@ -733,12 +781,30 @@ public void doAction(BalanceAction action) { replaceRegion(regionsPerServer[a.getFromServer()], a.getFromRegion(), a.getToRegion()); regionsPerServer[a.getToServer()] = replaceRegion(regionsPerServer[a.getToServer()], a.getToRegion(), a.getFromRegion()); - regionMoved(a.getFromRegion(), a.getFromServer(), a.getToServer()); - regionMoved(a.getToRegion(), a.getToServer(), a.getFromServer()); - break; + return ImmutableList.of(regionMoved(a.getFromRegion(), a.getFromServer(), a.getToServer()), + regionMoved(a.getToRegion(), a.getToServer(), a.getFromServer())); + case MOVE_BATCH: + assert action instanceof MoveBatchAction : action.getClass(); + MoveBatchAction mba = (MoveBatchAction) action; + List mbRegionPlans = new ArrayList<>(); + for (int serverIndex : mba.getServerToRegionsToRemove().keySet()) { + Set regionsToRemove = mba.getServerToRegionsToRemove().get(serverIndex); + regionsPerServer[serverIndex] = + removeRegions(regionsPerServer[serverIndex], regionsToRemove); + } + for (int serverIndex : mba.getServerToRegionsToAdd().keySet()) { + Set regionsToAdd = mba.getServerToRegionsToAdd().get(serverIndex); + regionsPerServer[serverIndex] = addRegions(regionsPerServer[serverIndex], regionsToAdd); + } + for (MoveRegionAction moveRegionAction : mba.getMoveActions()) { + mbRegionPlans.add(regionMoved(moveRegionAction.getRegion(), + moveRegionAction.getFromServer(), moveRegionAction.getToServer())); + } + return mbRegionPlans; default: - throw new RuntimeException("Uknown action:" + action.getType()); + throw new RuntimeException("Unknown action:" + action.getType()); } + return Collections.emptyList(); } /** @@ -822,7 +888,7 @@ void doAssignRegion(RegionInfo regionInfo, ServerName serverName) { doAction(new AssignRegionAction(region, server)); } - void regionMoved(int region, int oldServer, int newServer) { + RegionPlan regionMoved(int region, int oldServer, int newServer) { regionIndexToServerIndex[region] = newServer; if (initialRegionIndexToServerIndex[region] == newServer) { numMovedRegions--; // region moved back to original location @@ -853,6 +919,11 @@ void regionMoved(int region, int oldServer, int newServer) { updateForLocation(serverIndexToRackIndex, regionsPerRack, colocatedReplicaCountsPerRack, oldServer, newServer, primary, region); } + + // old server name can be null + ServerName oldServerName = oldServer == -1 ? null : servers[oldServer]; + + return new RegionPlan(regions[region], oldServerName, servers[newServer]); } /** @@ -899,6 +970,48 @@ int[] addRegion(int[] regions, int regionIndex) { return newRegions; } + int[] removeRegions(int[] regions, Set 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"); + } + + int[] newRegions = new int[newSize]; + int newIndex = 0; + + // Copy only the regions not in the removal set + for (int region : regions) { + if (!regionIndicesToRemove.contains(region)) { + newRegions[newIndex++] = region; + } + } + + // If the newIndex is smaller than newSize, some regions were missing from the input array + if (newIndex != newSize) { + throw new IllegalStateException("Region indices mismatch: some regions in the removal " + + "set were not found in the regions array"); + } + + return newRegions; + } + + int[] addRegions(int[] regions, Set regionIndicesToAdd) { + int[] newRegions = new int[regions.length + regionIndicesToAdd.size()]; + + // Copy the existing regions to the new array + System.arraycopy(regions, 0, newRegions, 0, regions.length); + + // Add the new regions at the end of the array + int newIndex = regions.length; + for (int regionIndex : regionIndicesToAdd) { + newRegions[newIndex++] = regionIndex; + } + + return newRegions; + } + int[] addRegionSorted(int[] regions, int regionIndex) { int[] newRegions = new int[regions.length + 1]; int i = 0; @@ -998,6 +1111,10 @@ void setNumMovedRegions(int numMovedRegions) { this.numMovedRegions = numMovedRegions; } + List getShuffledServerIndices() { + return shuffledServerIndicesSupplier.get(); + } + @Override public String toString() { StringBuilder desc = new StringBuilder("Cluster={servers=["); diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionals.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionals.java new file mode 100644 index 000000000000..70d335bac940 --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionals.java @@ -0,0 +1,230 @@ +/* + * 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.Set; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configurable; +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 + * (without needing to manage distinct RegionServer groups). Use of conditionals may cause an + * extremely unbalanced cluster to exceed its max balancer runtime. This is necessary because + * conditional candidate generation is quite expensive, and cutting it off early could prevent us + * from finding a solution. + */ +@InterfaceAudience.Private +final class BalancerConditionals implements Configurable { + + 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_KEY = + "hbase.master.balancer.stochastic.conditionals.distributeReplicas"; + public static final boolean DISTRIBUTE_REPLICAS_DEFAULT = false; + + public static final String ADDITIONAL_CONDITIONALS_KEY = + "hbase.master.balancer.stochastic.additionalConditionals"; + + private Set> conditionalClasses = Collections.emptySet(); + private Set conditionals = Collections.emptySet(); + private Configuration conf; + + private BalancerConditionals() { + } + + boolean shouldRunBalancer(BalancerClusterState cluster) { + return isConditionalBalancingEnabled() && conditionals.stream() + .map(RegionPlanConditional::getCandidateGenerators).flatMap(Collection::stream) + .map(generator -> generator.getWeight(cluster)).anyMatch(weight -> weight > 0); + } + + Set> getConditionalClasses() { + return Set.copyOf(conditionalClasses); + } + + Collection 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(); + } + + boolean isConditionalBalancingEnabled() { + return !conditionalClasses.isEmpty(); + } + + void clearConditionalWeightCaches() { + conditionals.stream().map(RegionPlanConditional::getCandidateGenerators) + .flatMap(Collection::stream) + .forEach(RegionPlanConditionalCandidateGenerator::clearWeightCache); + } + + void loadClusterState(BalancerClusterState cluster) { + conditionals = conditionalClasses.stream().map(clazz -> createConditional(clazz, conf, cluster)) + .filter(Objects::nonNull).collect(Collectors.toSet()); + } + + /** + * Indicates whether the action is good for our conditional compliance. + * @param cluster The cluster state + * @param action The proposed action + * @return -1 if conditionals improve, 0 if neutral, 1 if conditionals degrade + */ + int getViolationCountChange(BalancerClusterState cluster, BalanceAction action) { + boolean isViolatingPre = isViolating(cluster, action.undoAction()); + boolean isViolatingPost = isViolating(cluster, action); + if (isViolatingPre && isViolatingPost) { + return 0; + } else if (!isViolatingPre && isViolatingPost) { + return 1; + } else { + return -1; + } + } + + /** + * Check if the proposed action violates conditionals + * @param cluster The cluster state + * @param action The proposed action + */ + boolean isViolating(BalancerClusterState cluster, BalanceAction action) { + conditionals.forEach(conditional -> conditional.refreshClusterState(cluster)); + if (conditionals.isEmpty()) { + return false; + } + List regionPlans = cluster.convertActionToPlans(action); + for (RegionPlan regionPlan : regionPlans) { + if (isViolating(regionPlan)) { + return true; + } + } + return false; + } + + private boolean isViolating(RegionPlan regionPlan) { + for (RegionPlanConditional conditional : conditionals) { + if (conditional.isViolating(regionPlan)) { + return true; + } + } + return false; + } + + private RegionPlanConditional createConditional(Class clazz, + Configuration conf, BalancerClusterState cluster) { + if (conf == null) { + conf = new Configuration(); + } + if (cluster == null) { + cluster = new BalancerClusterState(Collections.emptyMap(), null, null, null, null); + } + try { + Constructor ctor = + clazz.getDeclaredConstructor(Configuration.class, BalancerClusterState.class); + return ReflectionUtils.instantiate(clazz.getName(), ctor, conf, cluster); + } catch (NoSuchMethodException e) { + LOG.warn("Cannot find constructor with Configuration and " + + "BalancerClusterState parameters for class '{}': {}", clazz.getName(), e.getMessage()); + } + return null; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + ImmutableSet.Builder> conditionalClasses = + ImmutableSet.builder(); + + boolean isolateSystemTables = + conf.getBoolean(ISOLATE_SYSTEM_TABLES_KEY, ISOLATE_SYSTEM_TABLES_DEFAULT); + if (isolateSystemTables) { + conditionalClasses.add(SystemTableIsolationConditional.class); + } + + boolean isolateMetaTable = conf.getBoolean(ISOLATE_META_TABLE_KEY, ISOLATE_META_TABLE_DEFAULT); + if (isolateMetaTable) { + conditionalClasses.add(MetaTableIsolationConditional.class); + } + + boolean distributeReplicas = + conf.getBoolean(DISTRIBUTE_REPLICAS_KEY, DISTRIBUTE_REPLICAS_DEFAULT); + if (distributeReplicas) { + conditionalClasses.add(DistributeReplicasConditional.class); + } + + Class[] classes = conf.getClasses(ADDITIONAL_CONDITIONALS_KEY); + for (Class clazz : classes) { + if (!RegionPlanConditional.class.isAssignableFrom(clazz)) { + LOG.warn("Class {} is not a RegionPlanConditional", clazz.getName()); + continue; + } + conditionalClasses.add(clazz.asSubclass(RegionPlanConditional.class)); + } + this.conditionalClasses = conditionalClasses.build(); + loadClusterState(null); + } + + @Override + public Configuration getConf() { + return conf; + } +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CacheAwareLoadBalancer.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CacheAwareLoadBalancer.java index bc31a317aebd..9fe23503dfd1 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CacheAwareLoadBalancer.java +++ b/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 createCandidateGenerators() { - List candidateGenerators = new ArrayList<>(2); - candidateGenerators.add(GeneratorFunctionType.LOAD.ordinal(), + protected Map, CandidateGenerator> + createCandidateGenerators() { + Map, CandidateGenerator> candidateGenerators = + new HashMap<>(2); + candidateGenerators.put(CacheAwareSkewnessCandidateGenerator.class, new CacheAwareSkewnessCandidateGenerator()); - candidateGenerators.add(GeneratorFunctionType.CACHE_RATIO.ordinal(), - new CacheAwareCandidateGenerator()); + candidateGenerators.put(CacheAwareCandidateGenerator.class, new CacheAwareCandidateGenerator()); return candidateGenerators; } @@ -409,8 +410,9 @@ protected void regionMoved(int region, int oldServer, int newServer) { }); } - public final void updateWeight(double[] weights) { - weights[GeneratorFunctionType.LOAD.ordinal()] += cost(); + @Override + public final void updateWeight(Map, Double> weights) { + weights.merge(LoadCandidateGenerator.class, cost(), Double::sum); } } @@ -478,8 +480,8 @@ private int getServerWithBestCacheRatioForRegion(int region) { } @Override - public final void updateWeight(double[] weights) { - weights[GeneratorFunctionType.CACHE_RATIO.ordinal()] += cost(); + public final void updateWeight(Map, Double> weights) { + weights.merge(CacheAwareCandidateGenerator.class, cost(), Double::sum); } } } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CandidateGenerator.java index d9245495e204..305470285381 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CandidateGenerator.java +++ b/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 = 1.0; + abstract BalanceAction generate(BalancerClusterState cluster); /** diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CostFunction.java index 5cc98478f9b9..bd99a31bec98 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CostFunction.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.master.balancer; +import java.util.Map; import org.apache.yetus.audience.InterfaceAudience; /** @@ -73,8 +74,15 @@ void postAction(BalanceAction action) { regionMoved(a.getFromRegion(), a.getFromServer(), a.getToServer()); regionMoved(a.getToRegion(), a.getToServer(), a.getFromServer()); break; + case MOVE_BATCH: + MoveBatchAction mba = (MoveBatchAction) action; + for (MoveRegionAction moveRegionAction : mba.getMoveActions()) { + regionMoved(moveRegionAction.getRegion(), moveRegionAction.getFromServer(), + moveRegionAction.getToServer()); + } + break; default: - throw new RuntimeException("Uknown action:" + action.getType()); + throw new RuntimeException("Unknown action:" + action.getType()); } } @@ -89,8 +97,8 @@ protected void regionMoved(int region, int oldServer, int newServer) { * Called once per init or after postAction. * @param weights the weights for every generator. */ - public void updateWeight(double[] weights) { - weights[StochasticLoadBalancer.GeneratorType.RANDOM.ordinal()] += cost(); + public void updateWeight(Map, Double> weights) { + weights.merge(RandomCandidateGenerator.class, cost(), Double::sum); } /** diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/DistributeReplicasCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/DistributeReplicasCandidateGenerator.java new file mode 100644 index 000000000000..a98fbee2101b --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/DistributeReplicasCandidateGenerator.java @@ -0,0 +1,120 @@ +/* + * 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.DistributeReplicasConditional.getReplicaKey; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * CandidateGenerator to distribute colocated replicas across different servers. + */ +@InterfaceAudience.Private +final class DistributeReplicasCandidateGenerator extends RegionPlanConditionalCandidateGenerator { + + static DistributeReplicasCandidateGenerator INSTANCE = new DistributeReplicasCandidateGenerator(); + + private static final Logger LOG = + LoggerFactory.getLogger(DistributeReplicasCandidateGenerator.class); + private static final int BATCH_SIZE = 100_000; + + private DistributeReplicasCandidateGenerator() { + } + + /** + * Generates a balancing action to distribute colocated replicas. Moves one replica of a colocated + * region to a different server. + * @param cluster Current state of the cluster. + * @param isWeighing Flag indicating if the generator is being used for weighing. + * @return A BalanceAction to move a replica or NULL_ACTION if no action is needed. + */ + @Override + BalanceAction generateCandidate(BalancerClusterState cluster, boolean isWeighing) { + return generateCandidate(cluster, isWeighing, false); + } + + BalanceAction generateCandidate(BalancerClusterState cluster, boolean isWeighing, + boolean isForced) { + // Iterate through shuffled servers to find colocated replicas + boolean foundColocatedReplicas = false; + List moveRegionActions = new ArrayList<>(); + for (int sourceIndex : cluster.getShuffledServerIndices()) { + int[] serverRegions = cluster.regionsPerServer[sourceIndex]; + Set replicaKeys = + new HashSet<>(serverRegions.length); + for (int regionIndex : serverRegions) { + DistributeReplicasConditional.ReplicaKey replicaKey = + getReplicaKey(cluster.regions[regionIndex]); + if (replicaKeys.contains(replicaKey)) { + foundColocatedReplicas = true; + if (isWeighing) { + // If weighing, fast exit with an actionable move + return getAction(sourceIndex, regionIndex, pickOtherRandomServer(cluster, sourceIndex), + -1); + } else { + // If not weighing, pick a good move + for (int i = 0; i < cluster.numServers; i++) { + // Randomize destination ordering so we aren't overloading one destination + int destinationIndex = pickOtherRandomServer(cluster, sourceIndex); + if (destinationIndex == sourceIndex) { + continue; + } + MoveRegionAction possibleAction = + new MoveRegionAction(regionIndex, sourceIndex, destinationIndex); + if (isForced) { + return possibleAction; + } else if (willBeAccepted(cluster, possibleAction)) { + cluster.doAction(possibleAction); // Update cluster state to reflect move + moveRegionActions.add(possibleAction); + break; + } + } + } + } else { + replicaKeys.add(replicaKey); + } + if (moveRegionActions.size() >= BATCH_SIZE) { + break; + } + } + if (moveRegionActions.size() >= BATCH_SIZE) { + break; + } + } + + if (!moveRegionActions.isEmpty()) { + MoveBatchAction batchAction = new MoveBatchAction(moveRegionActions); + undoBatchAction(cluster, batchAction); // Reset cluster state to before batch + return batchAction; + } + // If no colocated replicas are found, return NULL_ACTION + if (foundColocatedReplicas) { + LOG.warn("Could not find a place to put a colocated replica! We will force a move."); + return generateCandidate(cluster, isWeighing, true); + } else { + LOG.trace("No colocated replicas found. No balancing action required."); + } + return BalanceAction.NULL_ACTION; + } +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/DistributeReplicasConditional.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/DistributeReplicasConditional.java new file mode 100644 index 000000000000..01d7af500176 --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/DistributeReplicasConditional.java @@ -0,0 +1,175 @@ +/* + * 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.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionReplicaUtil; +import org.apache.hadoop.hbase.master.RegionPlan; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; +import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader; +import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache; + +/** + * If enabled, this class will help the balancer ensure that replicas aren't placed on the same + * servers or racks as their primary. Configure this via + * {@link BalancerConditionals#DISTRIBUTE_REPLICAS_KEY} + */ +@InterfaceAudience.Private +public class DistributeReplicasConditional extends RegionPlanConditional { + + /** + * Local mini cluster tests are only run on one host/rack by design. If enabled, this will pretend + * that localhost RegionServer threads are actually running on separate hosts/racks. This should + * only be used in unit tests. + */ + public static final String TEST_MODE_ENABLED_KEY = + "hbase.replica.distribution.conditional.testModeEnabled"; + + private static final Logger LOG = LoggerFactory.getLogger(DistributeReplicasConditional.class); + private static final LoadingCache REPLICA_KEY_CACHE = + CacheBuilder.newBuilder().maximumSize(1000).expireAfterAccess(Duration.ofMinutes(5)) + .build(new CacheLoader() { + @Override + public ReplicaKey load(RegionInfo region) { + return new ReplicaKey(region); + } + }); + + private final boolean isTestModeEnabled; + + public DistributeReplicasConditional(Configuration conf, BalancerClusterState cluster) { + super(conf, cluster); + this.isTestModeEnabled = conf.getBoolean(TEST_MODE_ENABLED_KEY, false); + } + + static ReplicaKey getReplicaKey(RegionInfo regionInfo) { + return REPLICA_KEY_CACHE.getUnchecked(regionInfo); + } + + @Override + public ValidationLevel getValidationLevel() { + if (isTestModeEnabled) { + return ValidationLevel.SERVER; + } + return ValidationLevel.RACK; + } + + @Override + List getCandidateGenerators() { + return Collections.singletonList(DistributeReplicasCandidateGenerator.INSTANCE); + } + + @Override + boolean isViolatingServer(RegionPlan regionPlan, Set serverRegions) { + return checkViolation(regionPlan.getRegionInfo(), getReplicaKey(regionPlan.getRegionInfo()), + serverRegions); + } + + @Override + boolean isViolatingHost(RegionPlan regionPlan, Set hostRegions) { + return checkViolation(regionPlan.getRegionInfo(), getReplicaKey(regionPlan.getRegionInfo()), + hostRegions); + } + + @Override + boolean isViolatingRack(RegionPlan regionPlan, Set rackRegions) { + return checkViolation(regionPlan.getRegionInfo(), getReplicaKey(regionPlan.getRegionInfo()), + rackRegions); + } + + private boolean checkViolation(RegionInfo movingRegion, ReplicaKey movingReplicaKey, + Set destinationRegions) { + for (RegionInfo regionInfo : destinationRegions) { + if (regionInfo.equals(movingRegion)) { + continue; + } + if (getReplicaKey(regionInfo).equals(movingReplicaKey)) { + return true; + } + } + return false; + } + + /** + * This is necessary because it would be too expensive to use + * {@link RegionReplicaUtil#isReplicasForSameRegion(RegionInfo, RegionInfo)} for every combo of + * regions. + */ + static class ReplicaKey { + private final Pair startAndStopKeys; + + ReplicaKey(RegionInfo regionInfo) { + this.startAndStopKeys = new Pair<>(new ByteArrayWrapper(regionInfo.getStartKey()), + new ByteArrayWrapper(regionInfo.getEndKey())); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ReplicaKey)) { + return false; + } + ReplicaKey other = (ReplicaKey) o; + return this.startAndStopKeys.equals(other.startAndStopKeys); + } + + @Override + public int hashCode() { + return startAndStopKeys.hashCode(); + } + } + + static class ByteArrayWrapper { + private final byte[] bytes; + + ByteArrayWrapper(byte[] prefix) { + this.bytes = Arrays.copyOf(prefix, prefix.length); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ByteArrayWrapper)) { + return false; + } + ByteArrayWrapper other = (ByteArrayWrapper) o; + return Arrays.equals(this.bytes, other.bytes); + } + + @Override + public int hashCode() { + return Arrays.hashCode(bytes); + } + } + +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java index fbe91a921daa..657410f0aeb4 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java @@ -80,17 +80,19 @@ public void setFavoredNodesManager(FavoredNodesManager fnm) { } @Override - protected List createCandidateGenerators() { - List fnPickers = new ArrayList<>(2); - fnPickers.add(new FavoredNodeLoadPicker()); - fnPickers.add(new FavoredNodeLocalityPicker()); + protected Map, CandidateGenerator> + createCandidateGenerators() { + Map, CandidateGenerator> fnPickers = new HashMap<>(2); + fnPickers.put(FavoredNodeLoadPicker.class, new FavoredNodeLoadPicker()); + fnPickers.put(FavoredNodeLocalityPicker.class, new FavoredNodeLocalityPicker()); return fnPickers; } /** Returns any candidate generator in random */ @Override - protected CandidateGenerator getRandomGenerator() { - return candidateGenerators.get(ThreadLocalRandom.current().nextInt(candidateGenerators.size())); + protected CandidateGenerator getRandomGenerator(BalancerClusterState cluster) { + return candidateGenerators.values().stream().toList() + .get(ThreadLocalRandom.current().nextInt(candidateGenerators.size())); } /** diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/LocalityBasedCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/LocalityBasedCostFunction.java index 678c9a3e9adf..fc2d6cc12fb3 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/LocalityBasedCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/LocalityBasedCostFunction.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.master.balancer; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.master.balancer.BalancerClusterState.LocalityType; import org.apache.yetus.audience.InterfaceAudience; @@ -89,7 +90,7 @@ private double getWeightedLocality(int region, int entity) { } @Override - public final void updateWeight(double[] weights) { - weights[StochasticLoadBalancer.GeneratorType.LOCALITY.ordinal()] += cost(); + public final void updateWeight(Map, Double> weights) { + weights.merge(LocalityBasedCandidateGenerator.class, cost(), Double::sum); } } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/MetaTableIsolationCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/MetaTableIsolationCandidateGenerator.java new file mode 100644 index 000000000000..8160f4cf677b --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/MetaTableIsolationCandidateGenerator.java @@ -0,0 +1,35 @@ +/* + * 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 org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public final class MetaTableIsolationCandidateGenerator extends TableIsolationCandidateGenerator { + + static MetaTableIsolationCandidateGenerator INSTANCE = new MetaTableIsolationCandidateGenerator(); + + private MetaTableIsolationCandidateGenerator() { + } + + @Override + boolean shouldBeIsolated(RegionInfo regionInfo) { + return regionInfo.isMetaRegion(); + } +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/MetaTableIsolationConditional.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/MetaTableIsolationConditional.java new file mode 100644 index 000000000000..41a22bad88ee --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/MetaTableIsolationConditional.java @@ -0,0 +1,70 @@ +/* + * 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.List; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.master.RegionPlan; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; + +/** + * If enabled, this class will help the balancer ensure that the meta table lives on its own + * RegionServer. Configure this via {@link BalancerConditionals#ISOLATE_META_TABLE_KEY} + */ +@InterfaceAudience.Private +class MetaTableIsolationConditional extends RegionPlanConditional { + + public MetaTableIsolationConditional(Configuration conf, BalancerClusterState cluster) { + super(conf, cluster); + } + + @Override + List getCandidateGenerators() { + return ImmutableList.of(MetaTableIsolationCandidateGenerator.INSTANCE, + new TableColocationCandidateGenerator(TableName.META_TABLE_NAME)); + } + + @Override + public boolean isViolatingServer(RegionPlan regionPlan, Set 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; + } + + private boolean isRegionToIsolate(RegionInfo regionInfo) { + return regionInfo.isMetaRegion(); + } +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/MoveBatchAction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/MoveBatchAction.java new file mode 100644 index 000000000000..9aded615a468 --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/MoveBatchAction.java @@ -0,0 +1,53 @@ +/* + * 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.List; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap; +import org.apache.hbase.thirdparty.com.google.common.collect.Multimaps; + +@InterfaceAudience.Private +public class MoveBatchAction extends BalanceAction { + private final List moveActions; + + MoveBatchAction(List moveActions) { + super(Type.MOVE_BATCH); + this.moveActions = moveActions; + } + + @Override + long getStepCount() { + return moveActions.size(); + } + + public HashMultimap getServerToRegionsToRemove() { + return moveActions.stream().collect(Multimaps.toMultimap(MoveRegionAction::getFromServer, + MoveRegionAction::getRegion, HashMultimap::create)); + } + + public HashMultimap getServerToRegionsToAdd() { + return moveActions.stream().collect(Multimaps.toMultimap(MoveRegionAction::getToServer, + MoveRegionAction::getRegion, HashMultimap::create)); + } + + List getMoveActions() { + return moveActions; + } +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionCountSkewCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionCountSkewCostFunction.java index 442bbc9b7bcf..669f1df8f4fd 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionCountSkewCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionCountSkewCostFunction.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.master.balancer; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.yetus.audience.InterfaceAudience; @@ -61,7 +62,7 @@ protected void regionMoved(int region, int oldServer, int newServer) { } @Override - public final void updateWeight(double[] weights) { - weights[StochasticLoadBalancer.GeneratorType.LOAD.ordinal()] += cost(); + public final void updateWeight(Map, Double> weights) { + weights.merge(LoadCandidateGenerator.class, cost(), Double::sum); } } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionPlanConditional.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionPlanConditional.java new file mode 100644 index 000000000000..1ad91a15f3fb --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionPlanConditional.java @@ -0,0 +1,125 @@ +/* + * 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.HashSet; +import java.util.List; +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; +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public abstract class RegionPlanConditional { + private static final Logger LOG = LoggerFactory.getLogger(RegionPlanConditional.class); + private BalancerClusterState cluster; + + RegionPlanConditional(Configuration conf, BalancerClusterState cluster) { + this.cluster = cluster; + } + + public enum ValidationLevel { + SERVER, // Just check server + HOST, // Check host and server + RACK // Check rack, host, and server + } + + public ValidationLevel getValidationLevel() { + return ValidationLevel.SERVER; + } + + void refreshClusterState(BalancerClusterState cluster) { + this.cluster = cluster; + } + + /** + * Get the candidate generator(s) for this conditional. This can be useful to provide the balancer + * with hints that will appease your conditional. Your conditionals will be triggered in order. + * @return the candidate generator for this conditional + */ + abstract List getCandidateGenerators(); + + /** + * Check if the conditional is violated by the given region plan. + * @param regionPlan the region plan to check + * @return true if the conditional is violated + */ + boolean isViolating(RegionPlan regionPlan) { + if (regionPlan == null) { + return false; + } + int destinationServerIdx = cluster.serversToIndex.get(regionPlan.getDestination().getAddress()); + + // Check Server + int[] destinationRegionIndices = cluster.regionsPerServer[destinationServerIdx]; + Set serverRegions = new HashSet<>(destinationRegionIndices.length); + for (int regionIdx : destinationRegionIndices) { + serverRegions.add(cluster.regions[regionIdx]); + } + if (isViolatingServer(regionPlan, serverRegions)) { + return true; + } + + if (getValidationLevel() == ValidationLevel.SERVER) { + return false; + } + + // Check Host + int hostIdx = cluster.serverIndexToHostIndex[destinationServerIdx]; + int[] hostRegionIndices = cluster.regionsPerHost[hostIdx]; + Set hostRegions = new HashSet<>(hostRegionIndices.length); + for (int regionIdx : hostRegionIndices) { + hostRegions.add(cluster.regions[regionIdx]); + } + if (isViolatingHost(regionPlan, hostRegions)) { + return true; + } + + if (getValidationLevel() == ValidationLevel.HOST) { + return false; + } + + // Check Rack + int rackIdx = cluster.serverIndexToRackIndex[destinationServerIdx]; + int[] rackRegionIndices = cluster.regionsPerRack[rackIdx]; + Set rackRegions = new HashSet<>(rackRegionIndices.length); + for (int regionIdx : rackRegionIndices) { + rackRegions.add(cluster.regions[regionIdx]); + } + if (isViolatingRack(regionPlan, rackRegions)) { + return true; + } + + return false; + } + + abstract boolean isViolatingServer(RegionPlan regionPlan, Set destinationRegions); + + boolean isViolatingHost(RegionPlan regionPlan, Set destinationRegions) { + return false; + } + + boolean isViolatingRack(RegionPlan regionPlan, Set destinationRegions) { + return false; + } +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionPlanConditionalCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionPlanConditionalCandidateGenerator.java new file mode 100644 index 000000000000..d0f91411cc55 --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionPlanConditionalCandidateGenerator.java @@ -0,0 +1,84 @@ +/* + * 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.time.Duration; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public abstract class RegionPlanConditionalCandidateGenerator extends CandidateGenerator { + + private static final Logger LOG = + LoggerFactory.getLogger(RegionPlanConditionalCandidateGenerator.class); + + private static final Duration WEIGHT_CACHE_TTL = Duration.ofMinutes(1); + private long lastWeighedAt = -1; + private double lastWeight = 0.0; + + abstract BalanceAction generateCandidate(BalancerClusterState cluster, boolean isWeighing); + + @Override + BalanceAction generate(BalancerClusterState cluster) { + BalanceAction balanceAction = generateCandidate(cluster, false); + if (!willBeAccepted(cluster, balanceAction)) { + LOG.debug("Generated action is not widely accepted by all conditionals. " + + "Likely we are finding our way out of a deadlock. balanceAction={}", balanceAction); + } + return balanceAction; + } + + boolean willBeAccepted(BalancerClusterState cluster, BalanceAction action) { + return !BalancerConditionals.INSTANCE.isViolating(cluster, action); + } + + void undoBatchAction(BalancerClusterState cluster, MoveBatchAction batchAction) { + for (int i = batchAction.getMoveActions().size() - 1; i >= 0; i--) { + MoveRegionAction action = batchAction.getMoveActions().get(i); + cluster.doAction(action.undoAction()); + } + } + + void clearWeightCache() { + lastWeighedAt = -1; + } + + double getWeight(BalancerClusterState cluster) { + boolean hasCandidate = false; + + // Candidate generation is expensive, so for re-weighing generators we will cache + // the value for a bit + if (System.currentTimeMillis() - lastWeighedAt < WEIGHT_CACHE_TTL.toMillis()) { + return lastWeight; + } else { + hasCandidate = generateCandidate(cluster, true) != BalanceAction.NULL_ACTION; + lastWeighedAt = System.currentTimeMillis(); + } + + if (hasCandidate) { + // If this generator has something to do, then it's important + lastWeight = CandidateGenerator.MAX_WEIGHT; + } else { + lastWeight = 0; + } + return lastWeight; + } +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionReplicaGroupingCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionReplicaGroupingCostFunction.java index f482ada22b29..43eaceb4f775 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionReplicaGroupingCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionReplicaGroupingCostFunction.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.master.balancer; +import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import org.agrona.collections.Hashing; import org.agrona.collections.Int2IntCounterMap; @@ -73,8 +74,8 @@ protected double cost() { } @Override - public final void updateWeight(double[] weights) { - weights[StochasticLoadBalancer.GeneratorType.RACK.ordinal()] += cost(); + public final void updateWeight(Map, Double> weights) { + weights.merge(RegionReplicaRackCandidateGenerator.class, cost(), Double::sum); } /** diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java index e5cd5446c5c8..58507a80f833 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java @@ -22,11 +22,13 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Deque; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics; @@ -48,6 +50,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.base.Suppliers; + /** *

* This is a best effort load balancer. Given a Cost function F(C) => x It will randomly try and @@ -147,7 +151,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { private double curOverallCost = 0d; private double[] tempFunctionCosts; private double[] curFunctionCosts; - private double[] weightsOfGenerators; // Keep locality based picker and cost function to alert them // when new services are offered @@ -157,14 +160,17 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { private RegionReplicaHostCostFunction regionReplicaHostCostFunction; private RegionReplicaRackCostFunction regionReplicaRackCostFunction; - protected List candidateGenerators; + protected Map, CandidateGenerator> candidateGenerators; + private Map, Double> weightsOfGenerators; + private final Supplier>> shuffledGeneratorClasses = + Suppliers.memoizeWithExpiration(() -> { + List> shuffled = + new ArrayList<>(candidateGenerators.keySet()); + Collections.shuffle(shuffled); + return shuffled; + }, 5, TimeUnit.SECONDS); - public enum GeneratorType { - RANDOM, - LOAD, - LOCALITY, - RACK - } + private final BalancerConditionals balancerConditionals = BalancerConditionals.INSTANCE; /** * The constructor that pass a MetricsStochasticBalancer to BaseLoadBalancer to replace its @@ -213,16 +219,18 @@ private void loadCustomCostFunctions(Configuration conf) { @RestrictedApi(explanation = "Should only be called in tests", link = "", allowedOnPath = ".*/src/test/.*") - List getCandidateGenerators() { + Map, CandidateGenerator> getCandidateGenerators() { return this.candidateGenerators; } - protected List createCandidateGenerators() { - List candidateGenerators = new ArrayList(4); - candidateGenerators.add(GeneratorType.RANDOM.ordinal(), new RandomCandidateGenerator()); - candidateGenerators.add(GeneratorType.LOAD.ordinal(), new LoadCandidateGenerator()); - candidateGenerators.add(GeneratorType.LOCALITY.ordinal(), localityCandidateGenerator); - candidateGenerators.add(GeneratorType.RACK.ordinal(), + protected Map, CandidateGenerator> + createCandidateGenerators() { + Map, CandidateGenerator> candidateGenerators = + new HashMap<>(4); + candidateGenerators.put(RandomCandidateGenerator.class, new RandomCandidateGenerator()); + candidateGenerators.put(LoadCandidateGenerator.class, new LoadCandidateGenerator()); + candidateGenerators.put(LocalityBasedCandidateGenerator.class, localityCandidateGenerator); + candidateGenerators.put(RegionReplicaCandidateGenerator.class, new RegionReplicaRackCandidateGenerator()); return candidateGenerators; } @@ -259,6 +267,8 @@ protected void loadConf(Configuration conf) { localityCost = new ServerLocalityCostFunction(conf); rackLocalityCost = new RackLocalityCostFunction(conf); + // Order is important here. We need to construct conditionals to load candidate generators + balancerConditionals.setConf(conf); this.candidateGenerators = createCandidateGenerators(); regionReplicaHostCostFunction = new RegionReplicaHostCostFunction(conf); @@ -332,8 +342,33 @@ void updateMetricsSize(int size) { } private boolean areSomeRegionReplicasColocated(BalancerClusterState c) { + if (!c.hasRegionReplicas || balancerConditionals.isReplicaDistributionEnabled()) { + // This check is unnecessary without replicas, or with conditional replica distribution + // The balancer will auto-run if conditional replica distribution candidates are available + return false; + } + + // Check host regionReplicaHostCostFunction.prepare(c); - return (Math.abs(regionReplicaHostCostFunction.cost()) > CostFunction.COST_EPSILON); + double hostCost = Math.abs(regionReplicaHostCostFunction.cost()); + boolean colocatedAtHost = hostCost > CostFunction.COST_EPSILON; + if (colocatedAtHost) { + return true; + } + LOG.trace("No host colocation detected with host cost={}", hostCost); + + // Check rack + regionReplicaRackCostFunction.prepare(c); + double rackCost = Math.abs(regionReplicaRackCostFunction.cost()); + boolean colocatedAtRack = + (Math.abs(regionReplicaRackCostFunction.cost()) > CostFunction.COST_EPSILON); + if (colocatedAtRack) { + return true; + } + LOG.trace("No rack colocation detected with rack cost={}", rackCost); + + return DistributeReplicasCandidateGenerator.INSTANCE.generateCandidate(c, true) + != BalanceAction.NULL_ACTION; } private String getBalanceReason(double total, double sumMultiplier) { @@ -372,21 +407,28 @@ boolean needsBalance(TableName tableName, BalancerClusterState cluster) { return true; } - if (sloppyRegionServerExist(cs)) { + if (!balancerConditionals.shouldSkipSloppyServerEvaluation() && sloppyRegionServerExist(cs)) { LOG.info("Running balancer because cluster has sloppy server(s)." + " function cost={}", functionCost()); return true; } + if (balancerConditionals.shouldRunBalancer(cluster)) { + LOG.info("Running balancer because conditional candidate generators have important moves"); + return true; + } + double total = 0.0; + double multiplierTotal = 0; // this.sumMultiplier is not necessarily initialized at this point for (CostFunction c : costFunctions) { if (!c.isNeeded()) { LOG.trace("{} not needed", c.getClass().getSimpleName()); continue; } total += c.cost() * c.getMultiplier(); + multiplierTotal += c.getMultiplier(); } - boolean balanced = (total / sumMultiplier < minCostNeedBalance); + boolean balanced = (total / multiplierTotal < minCostNeedBalance); if (balanced) { final double calculatedTotal = total; @@ -394,22 +436,26 @@ boolean needsBalance(TableName tableName, BalancerClusterState cluster) { costFunctions); LOG.info( "{} - skipping load balancing because weighted average imbalance={} <= " - + "threshold({}). If you want more aggressive balancing, either lower " + + "threshold({}) and conditionals do not have opinionated move candidates. " + + "consecutive balancer runs. If you want more aggressive balancing, either lower " + "hbase.master.balancer.stochastic.minCostNeedBalance from {} or increase the relative " + "multiplier(s) of the specific cost function(s). functionCost={}", isByTable ? "Table specific (" + tableName + ")" : "Cluster wide", total / sumMultiplier, minCostNeedBalance, minCostNeedBalance, functionCost()); } else { - LOG.info("{} - Calculating plan. may take up to {}ms to complete.", - isByTable ? "Table specific (" + tableName + ")" : "Cluster wide", maxRunningTime); + LOG.info( + "{} - Calculating plan. may take up to {}ms to complete. currentCost={}, targetCost={}", + isByTable ? "Table specific (" + tableName + ")" : "Cluster wide", maxRunningTime, total, + minCostNeedBalance); } return !balanced; } @RestrictedApi(explanation = "Should only be called in tests", link = "", allowedOnPath = ".*(/src/test/.*|StochasticLoadBalancer).java") - BalanceAction nextAction(BalancerClusterState cluster) { - return getRandomGenerator().generate(cluster); + Pair nextAction(BalancerClusterState cluster) { + CandidateGenerator generator = getRandomGenerator(cluster); + return Pair.newPair(generator, generator.generate(cluster)); } /** @@ -417,25 +463,69 @@ BalanceAction nextAction(BalancerClusterState cluster) { * 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() { - double sum = 0; - for (int i = 0; i < weightsOfGenerators.length; i++) { - sum += weightsOfGenerators[i]; - weightsOfGenerators[i] = sum; + protected CandidateGenerator getRandomGenerator(BalancerClusterState cluster) { + // Prefer conditional generators if they have moves to make + if (balancerConditionals.isConditionalBalancingEnabled()) { + for (RegionPlanConditional conditional : balancerConditionals.getConditionals()) { + List generators = + conditional.getCandidateGenerators(); + for (RegionPlanConditionalCandidateGenerator generator : generators) { + if (generator.getWeight(cluster) > 0) { + return generator; + } + } + } + } + + double rand = ThreadLocalRandom.current().nextDouble(); + if ( + !balancerConditionals.isReplicaDistributionEnabled() + && areSomeRegionReplicasColocated(cluster) + ) { + // If we aren't use conditional replica distribution, then we should ensure that + // the region replica cost functions' candidate generators are used often + if (rand < 0.25 && candidateGenerators.containsKey(RegionReplicaCandidateGenerator.class)) { + return candidateGenerators.get(RegionReplicaCandidateGenerator.class); + } + if ( + rand < 0.5 && candidateGenerators.containsKey(RegionReplicaRackCandidateGenerator.class) + ) { + return candidateGenerators.get(RegionReplicaRackCandidateGenerator.class); + } } - if (sum == 0) { - return candidateGenerators.get(0); + + List> generatorClasses = shuffledGeneratorClasses.get(); + List partialSums = new ArrayList<>(generatorClasses.size()); + double sum = 0.0; + for (Class clazz : generatorClasses) { + double weight = weightsOfGenerators.getOrDefault(clazz, 0.0); + sum += weight; + partialSums.add(sum); + } + + // If the sum of all weights is zero, fall back to a default (e.g., first in the list) + if (sum == 0.0) { + // If no generators at all, fail fast or throw + if (generatorClasses.isEmpty()) { + throw new IllegalStateException("No candidate generators available"); + } + return candidateGenerators.get(generatorClasses.get(0)); } - for (int i = 0; i < weightsOfGenerators.length; i++) { - weightsOfGenerators[i] /= sum; + + // Normalize partial sums so that the last one should be exactly 1.0 + for (int i = 0; i < partialSums.size(); i++) { + partialSums.set(i, partialSums.get(i) / sum); } - double rand = ThreadLocalRandom.current().nextDouble(); - for (int i = 0; i < weightsOfGenerators.length; i++) { - if (rand <= weightsOfGenerators[i]) { - return candidateGenerators.get(i); + + // Generate a random number and pick the first generator whose partial sum is >= rand + for (int i = 0; i < partialSums.size(); i++) { + if (rand <= partialSums.get(i)) { + return candidateGenerators.get(generatorClasses.get(i)); } } - return candidateGenerators.get(candidateGenerators.size() - 1); + + // Fallback: if for some reason we didn't return above, return the last generator + return candidateGenerators.get(generatorClasses.get(generatorClasses.size() - 1)); } @RestrictedApi(explanation = "Should only be called in tests", link = "", @@ -453,6 +543,7 @@ private long calculateMaxSteps(BalancerClusterState cluster) { * approach the optimal state given enough steps. */ @Override + @SuppressWarnings("checkstyle:MethodLength") protected List balanceTable(TableName tableName, Map> loadOfOneTable) { // On clusters with lots of HFileLinks or lots of reference files, @@ -473,6 +564,8 @@ protected List balanceTable(TableName tableName, long startTime = EnvironmentEdgeManager.currentTime(); initCosts(cluster); + balancerConditionals.loadClusterState(cluster); + balancerConditionals.clearConditionalWeightCaches(); sumMultiplier = 0; for (CostFunction c : costFunctions) { @@ -512,6 +605,7 @@ protected List balanceTable(TableName tableName, calculatedMaxSteps, maxSteps); } } + LOG.info( "Start StochasticLoadBalancer.balancer, initial weighted average imbalance={}, " + "functionCost={} computedMaxSteps={}", @@ -521,21 +615,70 @@ protected List balanceTable(TableName tableName, // Perform a stochastic walk to see if we can get a good fit. long step; + boolean planImprovedConditionals = false; + Map, Long> generatorToStepCount = new HashMap<>(); + Map, Long> generatorToApprovedActionCount = new HashMap<>(); for (step = 0; step < computedMaxSteps; step++) { - BalanceAction action = nextAction(cluster); + Pair nextAction = nextAction(cluster); + CandidateGenerator generator = nextAction.getFirst(); + BalanceAction action = nextAction.getSecond(); if (action.getType() == BalanceAction.Type.NULL) { continue; } - cluster.doAction(action); - updateCostsAndWeightsWithAction(cluster, action); + generatorToStepCount.merge(generator.getClass(), action.getStepCount(), Long::sum); + long additionalSteps = action.getStepCount() - 1; + if (additionalSteps > 0) { + step += additionalSteps; + } + int conditionalViolationsChange = 0; + boolean isViolatingConditionals = false; + boolean moveImprovedConditionals = false; + // Only check conditionals if they are enabled + if (balancerConditionals.isConditionalBalancingEnabled()) { + // Always accept a conditional generator output. Sometimes conditional generators + // may need to make controversial moves in order to break what would otherwise + // be a deadlocked situation. + // Otherwise, for normal moves, evaluate the action. + if (RegionPlanConditionalCandidateGenerator.class.isAssignableFrom(generator.getClass())) { + conditionalViolationsChange = -1; + } else { + conditionalViolationsChange = + balancerConditionals.getViolationCountChange(cluster, action); + isViolatingConditionals = balancerConditionals.isViolating(cluster, action); + } + moveImprovedConditionals = conditionalViolationsChange < 0; + if (moveImprovedConditionals) { + planImprovedConditionals = true; + } + } + + // Change state and evaluate costs + try { + cluster.doAction(action); + } catch (IllegalStateException | ArrayIndexOutOfBoundsException e) { + LOG.warn( + "Generator {} produced invalid action! " + + "Debug your candidate generator as this is likely a bug, " + + "and may cause a balancer deadlock. {}", + generator.getClass().getSimpleName(), action, e); + continue; + } + updateCostsAndWeightsWithAction(cluster, action); newCost = computeCost(cluster, currentCost); - // Should this be kept? - if (newCost < currentCost) { + boolean conditionalsSimilarCostsImproved = + (newCost < currentCost && conditionalViolationsChange == 0 && !isViolatingConditionals); + // Our first priority is to reduce conditional violations + // Our second priority is to reduce balancer cost + // change, regardless of cost change + if (moveImprovedConditionals || conditionalsSimilarCostsImproved) { currentCost = newCost; + generatorToApprovedActionCount.merge(generator.getClass(), action.getStepCount(), + Long::sum); + balancerConditionals.loadClusterState(cluster); // save for JMX curOverallCost = currentCost; @@ -554,9 +697,19 @@ protected List balanceTable(TableName tableName, } long endTime = EnvironmentEdgeManager.currentTime(); + // Build the log message + StringBuilder logMessage = new StringBuilder("CandidateGenerator activity summary:\n"); + generatorToStepCount.forEach((generator, count) -> { + long approvals = generatorToApprovedActionCount.getOrDefault(generator, 0L); + logMessage.append(String.format(" - %s: %d steps, %d approvals%n", generator.getSimpleName(), + count, approvals)); + }); + // Log the message + LOG.info(logMessage.toString()); + metricsBalancer.balanceCluster(endTime - startTime); - if (initCost > currentCost) { + if (planImprovedConditionals || initCost > currentCost) { updateStochasticCosts(tableName, curOverallCost, curFunctionCosts); List plans = createRegionPlans(cluster); LOG.info( @@ -571,7 +724,8 @@ protected List balanceTable(TableName tableName, } LOG.info( "Could not find a better moving plan. Tried {} different configurations in " - + "{} ms, and did not find anything with an imbalance score less than {}", + + "{} ms, and did not find anything with an imbalance score less than {} " + + "and could not improve conditional violations", step, endTime - startTime, initCost / sumMultiplier); return null; } @@ -747,7 +901,10 @@ private void updateRegionLoad() { allowedOnPath = ".*(/src/test/.*|StochasticLoadBalancer).java") void initCosts(BalancerClusterState cluster) { // Initialize the weights of generator every time - weightsOfGenerators = new double[this.candidateGenerators.size()]; + weightsOfGenerators = new HashMap<>(this.candidateGenerators.size()); + for (Class clazz : candidateGenerators.keySet()) { + weightsOfGenerators.put(clazz, 0.0); + } for (CostFunction c : costFunctions) { c.prepare(cluster); c.updateWeight(weightsOfGenerators); @@ -761,8 +918,8 @@ void initCosts(BalancerClusterState cluster) { allowedOnPath = ".*(/src/test/.*|StochasticLoadBalancer).java") void updateCostsAndWeightsWithAction(BalancerClusterState cluster, BalanceAction action) { // Reset all the weights to 0 - for (int i = 0; i < weightsOfGenerators.length; i++) { - weightsOfGenerators[i] = 0; + for (Class clazz : candidateGenerators.keySet()) { + weightsOfGenerators.put(clazz, 0.0); } for (CostFunction c : costFunctions) { if (c.isNeeded()) { diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/SystemTableIsolationCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/SystemTableIsolationCandidateGenerator.java new file mode 100644 index 000000000000..f71ce45039d7 --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/SystemTableIsolationCandidateGenerator.java @@ -0,0 +1,47 @@ +/* + * 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 org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public final class SystemTableIsolationCandidateGenerator extends TableIsolationCandidateGenerator { + + static SystemTableIsolationCandidateGenerator INSTANCE = + new SystemTableIsolationCandidateGenerator(); + + private boolean isolateMeta = false; + + private SystemTableIsolationCandidateGenerator() { + } + + void setIsolateMeta(boolean isolateMeta) { + this.isolateMeta = isolateMeta; + } + + @Override + boolean shouldBeIsolated(RegionInfo regionInfo) { + if (isolateMeta) { + return regionInfo.getTable().isSystemTable() && !regionInfo.isMetaRegion(); + } else { + return regionInfo.getTable().isSystemTable(); + } + } + +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/SystemTableIsolationConditional.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/SystemTableIsolationConditional.java new file mode 100644 index 000000000000..a361aa87ac88 --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/SystemTableIsolationConditional.java @@ -0,0 +1,91 @@ +/* + * 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.List; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +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} + */ +@InterfaceAudience.Private +class SystemTableIsolationConditional extends RegionPlanConditional { + + private final Set systemTables; + + public SystemTableIsolationConditional(Configuration conf, BalancerClusterState cluster) { + super(conf, cluster); + boolean isolateMeta = conf.getBoolean(BalancerConditionals.ISOLATE_META_TABLE_KEY, false); + SystemTableIsolationCandidateGenerator.INSTANCE.setIsolateMeta(isolateMeta); + systemTables = cluster.tables.stream().map(TableName::valueOf).filter(TableName::isSystemTable) + .filter(t -> !isolateMeta || !t.equals(TableName.META_TABLE_NAME)) + .collect(Collectors.toSet()); + } + + @Override + List getCandidateGenerators() { + List generators = + new ArrayList<>(systemTables.size() + 1); + generators.add(SystemTableIsolationCandidateGenerator.INSTANCE); + for (TableName systemTable : systemTables) { + generators.add(new TableColocationCandidateGenerator(systemTable)); + } + return generators; + } + + @Override + public boolean isViolatingServer(RegionPlan regionPlan, Set 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; + } + + private boolean isRegionToIsolate(RegionInfo regionInfo) { + boolean isRegionToIsolate = false; + if (BalancerConditionals.INSTANCE.isMetaTableIsolationEnabled() && regionInfo.isMetaRegion()) { + isRegionToIsolate = true; + } else if (regionInfo.getTable().isSystemTable()) { + isRegionToIsolate = true; + } + return isRegionToIsolate; + } + +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/TableColocationCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/TableColocationCandidateGenerator.java new file mode 100644 index 000000000000..2927a3ea16f6 --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/TableColocationCandidateGenerator.java @@ -0,0 +1,107 @@ +/* + * 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.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This will generate candidates that colocate a table on the number of RegionServers equal to its + * number of replicas. For example, this is useful when isolating system tables. + */ +@InterfaceAudience.Private +public final class TableColocationCandidateGenerator + extends RegionPlanConditionalCandidateGenerator { + + private static final Logger LOG = + LoggerFactory.getLogger(TableColocationCandidateGenerator.class); + + private final TableName tableName; + + TableColocationCandidateGenerator(TableName tableName) { + this.tableName = tableName; + } + + @Override + BalanceAction generateCandidate(BalancerClusterState cluster, boolean isWeighing) { + int maxReplicaId = 0; + Map regionIdxToServerIdx = new HashMap<>(); + for (RegionInfo region : cluster.regions) { + if (region.getTable().equals(tableName)) { + int regionIdx = cluster.regionsToIndex.get(region); + regionIdxToServerIdx.put(regionIdx, cluster.regionIndexToServerIndex[regionIdx]); + if (region.getReplicaId() > maxReplicaId) { + maxReplicaId = region.getReplicaId(); + } + } + } + if (regionIdxToServerIdx.isEmpty()) { + LOG.trace("No regions found for table {}", tableName.getNameAsString()); + return BalanceAction.NULL_ACTION; + } + int numReplicas = maxReplicaId + 1; + Set serversHostingTable = new HashSet<>(regionIdxToServerIdx.values()); + if (serversHostingTable.size() <= numReplicas) { + return BalanceAction.NULL_ACTION; + } + List desiredHosts = serversHostingTable.stream().sorted().limit(numReplicas).toList(); + Set serversToEvacuate = new HashSet<>(serversHostingTable); + LOG.trace("Moving {} regions off of {} and onto {}", tableName.getNameAsString(), + serversToEvacuate, desiredHosts); + serversToEvacuate.removeAll(desiredHosts); + List moves = new ArrayList<>(); + int i = 0; + for (Map.Entry regionAndServer : regionIdxToServerIdx.entrySet()) { + if (serversToEvacuate.contains(regionAndServer.getValue())) { + boolean accepted = false; + for (int j = 0; j < desiredHosts.size(); j++) { + int desiredHostKey = (i + j) % desiredHosts.size(); + MoveRegionAction mra = new MoveRegionAction(regionAndServer.getKey(), + regionAndServer.getValue(), desiredHosts.get(desiredHostKey)); + if (isWeighing) { + return mra; + } else if (willBeAccepted(cluster, mra)) { + moves.add(mra); + cluster.doAction(mra); + accepted = true; + break; + } + } + if (!accepted) { + LOG.warn( + "Could not find placement for region {} on table {} from server {} to desired hosts {}", + regionAndServer.getKey(), tableName.getNameAsString(), regionAndServer.getValue(), + desiredHosts); + } + i++; + } + } + MoveBatchAction mba = new MoveBatchAction(moves); + undoBatchAction(cluster, mba); + return mba; + } +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/TableIsolationCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/TableIsolationCandidateGenerator.java new file mode 100644 index 000000000000..1a269574a651 --- /dev/null +++ b/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; + +@InterfaceAudience.Private +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 moves = new ArrayList<>(); + for (int serverIdx : cluster.getShuffledServerIndices()) { + boolean hasRegionsToIsolate = false; + Set 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; + } + cluster.doAction(possibleMove); // Update cluster state to reflect move + moves.add(possibleMove); + break; + } + } + } + } + } + if (moves.isEmpty()) { + return BalanceAction.NULL_ACTION; + } else { + MoveBatchAction batchAction = new MoveBatchAction(moves); + undoBatchAction(cluster, batchAction); // Reset cluster state to before batch action + return batchAction; + } + } +} diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java index 4a996e7796f5..54729f88b70e 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java @@ -285,6 +285,9 @@ protected List convertToList(final Map balancedCluster) { + if (balancedCluster == null) { + return "null"; + } NavigableSet sorted = new TreeSet<>(balancedCluster); ServerAndLoad[] arr = sorted.toArray(new ServerAndLoad[sorted.size()]); StringBuilder sb = new StringBuilder(sorted.size() * 4 + 4); diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/CandidateGeneratorTestUtil.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/CandidateGeneratorTestUtil.java new file mode 100644 index 000000000000..6b0881b9f73d --- /dev/null +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/CandidateGeneratorTestUtil.java @@ -0,0 +1,337 @@ +/* + * 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.StochasticLoadBalancer.MAX_RUNNING_TIME_KEY; +import static org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer.MIN_COST_NEED_BALANCE_KEY; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Base64; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.master.RegionPlan; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class CandidateGeneratorTestUtil { + + private static final Logger LOG = LoggerFactory.getLogger(CandidateGeneratorTestUtil.class); + + private CandidateGeneratorTestUtil() { + } + + static void runBalancerToExhaustion(Configuration conf, + Map> serverToRegions, + Set> expectations, float targetMaxBalancerCost) { + // Do the full plan. We're testing with a lot of regions + conf.setBoolean("hbase.master.balancer.stochastic.runMaxSteps", true); + conf.setLong(MAX_RUNNING_TIME_KEY, 15000); + + conf.setFloat(MIN_COST_NEED_BALANCE_KEY, targetMaxBalancerCost); + + Set userTablesToBalance = + serverToRegions.entrySet().stream().map(Map.Entry::getValue).flatMap(Collection::stream) + .map(RegionInfo::getTable).filter(t -> !t.isSystemTable()).collect(Collectors.toSet()); + BalancerClusterState cluster = createMockBalancerClusterState(serverToRegions); + StochasticLoadBalancer stochasticLoadBalancer = buildStochasticLoadBalancer(cluster, conf); + printClusterDistribution(cluster, 0); + int balancerRuns = 0; + int actionsTaken = 0; + long balancingMillis = 0; + boolean isBalanced = false; + while (!isBalanced) { + balancerRuns++; + if (balancerRuns > 1000) { + throw new RuntimeException("Balancer failed to find balance & meet expectations"); + } + long start = System.currentTimeMillis(); + List regionPlans = + stochasticLoadBalancer.balanceCluster(partitionRegionsByTable(serverToRegions)); + balancingMillis += System.currentTimeMillis() - start; + actionsTaken++; + if (regionPlans != null) { + // Apply all plans to serverToRegions + for (RegionPlan rp : regionPlans) { + ServerName source = rp.getSource(); + ServerName dest = rp.getDestination(); + RegionInfo region = rp.getRegionInfo(); + + // Update serverToRegions + serverToRegions.get(source).remove(region); + serverToRegions.get(dest).add(region); + actionsTaken++; + } + + // Now rebuild cluster and balancer from updated serverToRegions + cluster = createMockBalancerClusterState(serverToRegions); + stochasticLoadBalancer = buildStochasticLoadBalancer(cluster, conf); + } + printClusterDistribution(cluster, actionsTaken); + isBalanced = true; + for (Function condition : expectations) { + // Check if we've met all expectations for the candidate generator + if (!condition.apply(cluster)) { + isBalanced = false; + break; + } + } + if (isBalanced) { + // Check if the balancer thinks we're done too + LOG.info("All balancer conditions passed. Checking if balancer thinks it's done."); + if (stochasticLoadBalancer.needsBalance(HConstants.ENSEMBLE_TABLE_NAME, cluster)) { + LOG.info("Balancer would still like to run"); + isBalanced = false; + } else { + LOG.info("Balancer is done"); + } + } + } + LOG.info("Balancing took {}sec", Duration.ofMillis(balancingMillis).toMinutes()); + } + + /** + * Prints the current cluster distribution of regions per table per server + */ + static void printClusterDistribution(BalancerClusterState cluster, long actionsTaken) { + LOG.info("=== Cluster Distribution after {} balancer actions taken ===", actionsTaken); + + for (int i = 0; i < cluster.numServers; i++) { + int[] regions = cluster.regionsPerServer[i]; + int regionCount = (regions == null) ? 0 : regions.length; + + LOG.info("Server {}: {} regions", cluster.servers[i].getServerName(), regionCount); + + if (regionCount > 0) { + Map tableRegionCounts = new HashMap<>(); + + for (int regionIndex : regions) { + RegionInfo regionInfo = cluster.regions[regionIndex]; + TableName tableName = regionInfo.getTable(); + tableRegionCounts.put(tableName, tableRegionCounts.getOrDefault(tableName, 0) + 1); + } + + tableRegionCounts + .forEach((table, count) -> LOG.info(" - Table {}: {} regions", table, count)); + } + } + + LOG.info("==========================================="); + } + + /** + * Partitions the given serverToRegions map by table The tables are derived from the RegionInfo + * objects found in serverToRegions. + * @param serverToRegions The map of servers to their assigned regions. + * @return A map of tables to their server-to-region assignments. + */ + public static Map>> + partitionRegionsByTable(Map> serverToRegions) { + + // First, gather all tables from the regions + Set allTables = new HashSet<>(); + for (List regions : serverToRegions.values()) { + for (RegionInfo region : regions) { + allTables.add(region.getTable()); + } + } + + Map>> tablesToServersToRegions = new HashMap<>(); + + // Initialize each table with all servers mapped to empty lists + for (TableName table : allTables) { + Map> serverMap = new HashMap<>(); + for (ServerName server : serverToRegions.keySet()) { + serverMap.put(server, new ArrayList<>()); + } + tablesToServersToRegions.put(table, serverMap); + } + + // Distribute regions to their respective tables + for (Map.Entry> serverAndRegions : serverToRegions.entrySet()) { + ServerName server = serverAndRegions.getKey(); + List regions = serverAndRegions.getValue(); + + for (RegionInfo region : regions) { + TableName regionTable = region.getTable(); + // Now we know for sure regionTable is in allTables + Map> tableServerMap = + tablesToServersToRegions.get(regionTable); + tableServerMap.get(server).add(region); + } + } + + return tablesToServersToRegions; + } + + static StochasticLoadBalancer buildStochasticLoadBalancer(BalancerClusterState cluster, + Configuration conf) { + StochasticLoadBalancer stochasticLoadBalancer = + new StochasticLoadBalancer(new DummyMetricsStochasticBalancer()); + stochasticLoadBalancer.setClusterInfoProvider(new DummyClusterInfoProvider(conf)); + stochasticLoadBalancer.loadConf(conf); + stochasticLoadBalancer.initCosts(cluster); + return stochasticLoadBalancer; + } + + static BalancerClusterState + createMockBalancerClusterState(Map> serverToRegions) { + return new BalancerClusterState(serverToRegions, null, null, null, null); + } + + /** + * Generic method to validate table isolation. + */ + static boolean isTableIsolated(BalancerClusterState cluster, TableName tableName, + String tableType) { + for (int i = 0; i < cluster.numServers; i++) { + int[] regionsOnServer = cluster.regionsPerServer[i]; + if (regionsOnServer == null || regionsOnServer.length == 0) { + continue; // Skip empty servers + } + + boolean hasTargetTableRegion = false; + boolean hasOtherTableRegion = false; + + for (int regionIndex : regionsOnServer) { + RegionInfo regionInfo = cluster.regions[regionIndex]; + if (regionInfo.getTable().equals(tableName)) { + hasTargetTableRegion = true; + } else { + hasOtherTableRegion = true; + } + + // If the target table and any other table are on the same server, isolation is violated + if (hasTargetTableRegion && hasOtherTableRegion) { + LOG.warn( + "Server {} has both {} table regions and other table regions, violating isolation.", + cluster.servers[i].getServerName(), tableType); + return false; + } + } + } + LOG.info("{} table isolation validation passed.", tableType); + return true; + } + + /** + * Validates that the table is "colocated" by checking that exactly the given numberOfReplicas + * servers host this table. Put differently, if numberOfReplicas = 3, we expect exactly 3 servers + * (and no more) to have at least one region of this table. + * @param cluster The current state of the cluster. + * @param tableName The table to validate. + * @param tableType A string identifier used in logging (e.g., "SYSTEM" or "USER"). + * @param numberOfReplicas The expected number of servers hosting this table. + * @return true if exactly numberOfReplicas servers host this table, false otherwise. + */ + static boolean isTableColocated(BalancerClusterState cluster, TableName tableName, + String tableType, int numberOfReplicas) { + // Count how many servers host at least one region for this table + int serversHostingThisTable = 0; + for (int serverIdx = 0; serverIdx < cluster.numServers; serverIdx++) { + int[] regionsOnServer = cluster.regionsPerServer[serverIdx]; + if (regionsOnServer == null || regionsOnServer.length == 0) { + continue; // skip empty server + } + + // Check if this server hosts any region of the target table + boolean foundRegionForTable = false; + for (int regionIndex : regionsOnServer) { + RegionInfo regionInfo = cluster.regions[regionIndex]; + if (regionInfo.getTable().equals(tableName)) { + foundRegionForTable = true; + break; + } + } + if (foundRegionForTable) { + serversHostingThisTable++; + } + } + + // Compare the number of servers hosting this table to the given numberOfReplicas + if (serversHostingThisTable == numberOfReplicas) { + LOG.info("Table {} ({}) is colocated: {} servers host it (expected {}).", tableName, + tableType, serversHostingThisTable, numberOfReplicas); + return true; + } else { + LOG.warn("Table {} ({}) is NOT colocated: {} servers host it (expected {}).", tableName, + tableType, serversHostingThisTable, numberOfReplicas); + return false; + } + } + + /** + * Validates that each replica is isolated from its others. Ensures that no server hosts more than + * one replica of the same region (i.e., regions with identical start and end keys). + * @param cluster The current state of the cluster. + * @return true if all replicas are properly isolated, false otherwise. + */ + static boolean areAllReplicasDistributed(BalancerClusterState cluster) { + // Iterate over each server + for (int[] regionsPerServer : cluster.regionsPerServer) { + if (regionsPerServer == null || regionsPerServer.length == 0) { + continue; // Skip empty servers + } + + Set foundKeys = new HashSet<>(); + for (int regionIndex : regionsPerServer) { + RegionInfo regionInfo = cluster.regions[regionIndex]; + DistributeReplicasConditional.ReplicaKey replicaKey = + new DistributeReplicasConditional.ReplicaKey(regionInfo); + if (foundKeys.contains(replicaKey)) { + // Violation: Multiple replicas of the same region on the same server + LOG.warn("Replica isolation violated: one server hosts multiple replicas of key [{}].", + generateRegionKey(regionInfo)); + return false; + } + + foundKeys.add(replicaKey); + } + } + + LOG.info( + "Replica isolation validation passed: No server hosts multiple replicas of the same region."); + return true; + } + + /** + * Generates a unique key for a region based on its start and end keys. This method ensures that + * regions with identical start and end keys have the same key. + * @param regionInfo The RegionInfo object. + * @return A string representing the unique key of the region. + */ + private static String generateRegionKey(RegionInfo regionInfo) { + // Using Base64 encoding for byte arrays to ensure uniqueness and readability + String startKey = Base64.getEncoder().encodeToString(regionInfo.getStartKey()); + String endKey = Base64.getEncoder().encodeToString(regionInfo.getEndKey()); + + return regionInfo.getTable().getNameAsString() + ":" + startKey + ":" + endKey; + } + +} diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerConditionals.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerConditionals.java new file mode 100644 index 000000000000..5703c5085c16 --- /dev/null +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerConditionals.java @@ -0,0 +1,118 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestBalancerConditionals extends BalancerTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestBalancerConditionals.class); + + private BalancerConditionals balancerConditionals; + private BalancerClusterState mockCluster; + + @Before + public void setUp() { + balancerConditionals = BalancerConditionals.INSTANCE; + mockCluster = mockCluster(new int[] { 0, 1, 2 }); + } + + @Test + public void testDefaultConfiguration() { + Configuration conf = new Configuration(); + balancerConditionals.setConf(conf); + balancerConditionals.loadClusterState(mockCluster); + + assertEquals("No conditionals should be loaded by default", 0, + balancerConditionals.getConditionalClasses().size()); + } + + @Test + public void testSystemTableIsolationConditionalEnabled() { + Configuration conf = new Configuration(); + conf.setBoolean(BalancerConditionals.ISOLATE_SYSTEM_TABLES_KEY, true); + + balancerConditionals.setConf(conf); + balancerConditionals.loadClusterState(mockCluster); + + assertTrue("SystemTableIsolationConditional should be active", + balancerConditionals.shouldSkipSloppyServerEvaluation()); + } + + @Test + public void testMetaTableIsolationConditionalEnabled() { + Configuration conf = new Configuration(); + conf.setBoolean(BalancerConditionals.ISOLATE_META_TABLE_KEY, true); + + balancerConditionals.setConf(conf); + balancerConditionals.loadClusterState(mockCluster); + + assertTrue("MetaTableIsolationConditional should be active", + balancerConditionals.shouldSkipSloppyServerEvaluation()); + } + + @Test + public void testCustomConditionalsViaConfiguration() { + Configuration conf = new Configuration(); + conf.set(BalancerConditionals.ADDITIONAL_CONDITIONALS_KEY, + MetaTableIsolationConditional.class.getName()); + + balancerConditionals.setConf(conf); + balancerConditionals.loadClusterState(mockCluster); + + assertTrue("Custom conditionals should be loaded", + balancerConditionals.shouldSkipSloppyServerEvaluation()); + } + + @Test + public void testInvalidCustomConditionalClass() { + Configuration conf = new Configuration(); + conf.set(BalancerConditionals.ADDITIONAL_CONDITIONALS_KEY, "java.lang.String"); + + balancerConditionals.setConf(conf); + balancerConditionals.loadClusterState(mockCluster); + + assertEquals("Invalid classes should not be loaded as conditionals", 0, + balancerConditionals.getConditionalClasses().size()); + } + + @Test + public void testShouldSkipSloppyServerEvaluationWithMixedConditionals() { + Configuration conf = new Configuration(); + conf.setBoolean(BalancerConditionals.ISOLATE_SYSTEM_TABLES_KEY, true); + conf.setBoolean(BalancerConditionals.ISOLATE_META_TABLE_KEY, true); + + balancerConditionals.setConf(conf); + balancerConditionals.loadClusterState(mockCluster); + + assertTrue("Sloppy server evaluation should be skipped with relevant conditionals", + balancerConditionals.shouldSkipSloppyServerEvaluation()); + } +} diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingConditionalReplicaDistribution.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingConditionalReplicaDistribution.java new file mode 100644 index 000000000000..8ee62aeb793f --- /dev/null +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingConditionalReplicaDistribution.java @@ -0,0 +1,115 @@ +/* + * 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.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.apache.hadoop.hbase.util.Bytes; +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 TestLargeClusterBalancingConditionalReplicaDistribution { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLargeClusterBalancingConditionalReplicaDistribution.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestLargeClusterBalancingConditionalReplicaDistribution.class); + + /** + * The scale of this test is equivalent to that of + * {@link TestStochasticLoadBalancerRegionReplicaLargeCluster}, demonstrating the performance + * improvements to be gained from using conditional replica distribution. + */ + private static final int NUM_SERVERS = 1000; + private static final int NUM_REGIONS = 20_000; + private static final int NUM_REPLICAS = 3; + private static final int NUM_TABLES = 100; + + private static final ServerName[] servers = new ServerName[NUM_SERVERS]; + private static final Map> serverToRegions = new HashMap<>(); + + @BeforeClass + public static void setup() { + // Initialize servers + for (int i = 0; i < NUM_SERVERS; i++) { + servers[i] = ServerName.valueOf("server" + i, i, System.currentTimeMillis()); + serverToRegions.put(servers[i], new ArrayList<>()); + } + + // Create primary regions and their replicas + List allRegions = new ArrayList<>(); + for (int i = 0; i < NUM_REGIONS; i++) { + TableName tableName = getTableName(i); + // Define startKey and endKey for the region + byte[] startKey = Bytes.toBytes(i); + byte[] endKey = Bytes.toBytes(i + 1); + + // Create 3 replicas for each primary region + for (int replicaId = 0; replicaId < NUM_REPLICAS; replicaId++) { + RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey) + .setEndKey(endKey).setReplicaId(replicaId).build(); + allRegions.add(regionInfo); + } + } + + // Assign all regions to one server + for (RegionInfo regionInfo : allRegions) { + serverToRegions.get(servers[0]).add(regionInfo); + } + } + + private static TableName getTableName(int i) { + return TableName.valueOf("userTable" + i % NUM_TABLES); + } + + @Test + public void testReplicaDistribution() { + Configuration conf = new Configuration(false); + conf.setBoolean(BalancerConditionals.DISTRIBUTE_REPLICAS_KEY, true); + conf.setBoolean(DistributeReplicasConditional.TEST_MODE_ENABLED_KEY, true); + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 30_000); + + // turn off replica cost functions + conf.setLong("hbase.master.balancer.stochastic.regionReplicaRackCostKey", 0); + conf.setLong("hbase.master.balancer.stochastic.regionReplicaHostCostKey", 0); + + runBalancerToExhaustion(conf, serverToRegions, + Set.of(CandidateGeneratorTestUtil::areAllReplicasDistributed), 10.0f); + LOG.info("Meta table and system table regions are successfully isolated, " + + "meanwhile region replicas are appropriately distributed across RegionServers."); + } +} diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java new file mode 100644 index 000000000000..a6cebdf1baab --- /dev/null +++ b/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 { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLargeClusterBalancingMetaTableIsolation.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestLargeClusterBalancingMetaTableIsolation.class); + + private static final TableName META_TABLE_NAME = TableName.valueOf("hbase:meta"); + private static final TableName NON_META_TABLE_NAME = TableName.valueOf("userTable"); + + private static final int NUM_SERVERS = 100; + private static final int NUM_REGIONS = 10_000; + + private static final ServerName[] servers = new ServerName[NUM_SERVERS]; + private static final Map> serverToRegions = new HashMap<>(); + + @BeforeClass + public static void setup() { + // Initialize servers + for (int i = 0; i < NUM_SERVERS; i++) { + servers[i] = ServerName.valueOf("server" + i, i, System.currentTimeMillis()); + } + + // Create regions + List allRegions = new ArrayList<>(); + for (int i = 0; i < NUM_REGIONS; i++) { + TableName tableName = i < 3 ? META_TABLE_NAME : NON_META_TABLE_NAME; + byte[] startKey = new byte[1]; + startKey[0] = (byte) i; + byte[] endKey = new byte[1]; + endKey[0] = (byte) (i + 1); + + RegionInfo regionInfo = + RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).setEndKey(endKey).build(); + allRegions.add(regionInfo); + } + + // Assign all regions to the first server + serverToRegions.put(servers[0], new ArrayList<>(allRegions)); + for (int i = 1; i < NUM_SERVERS; i++) { + serverToRegions.put(servers[i], new ArrayList<>()); + } + } + + @Test + public void testMetaTableIsolation() { + Configuration conf = new Configuration(false); + conf.setBoolean(BalancerConditionals.ISOLATE_META_TABLE_KEY, true); + runBalancerToExhaustion(conf, serverToRegions, Set.of(this::isMetaTableIsolated), 0.1f); + } + + private boolean isMetaTableIsolated(BalancerClusterState cluster) { + return isTableIsolated(cluster, META_TABLE_NAME, "Meta"); + } + +} diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMultiTableIsolation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMultiTableIsolation.java new file mode 100644 index 000000000000..a8ba457b3123 --- /dev/null +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMultiTableIsolation.java @@ -0,0 +1,132 @@ +/* + * 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.isTableColocated; +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 TestLargeClusterBalancingMultiTableIsolation { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLargeClusterBalancingMultiTableIsolation.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestLargeClusterBalancingMultiTableIsolation.class); + + private static final TableName META_TABLE_NAME = TableName.valueOf("hbase:meta"); + private static final TableName SYSTEM_TABLE_NAME = TableName.valueOf("hbase:system"); + private static final TableName NON_ISOLATED_TABLE_NAME = TableName.valueOf("userTable"); + + private static final int NUM_SERVERS = 100; + private static final int NUM_REGIONS = 10_000; + + private static final ServerName[] servers = new ServerName[NUM_SERVERS]; + private static final Map> serverToRegions = new HashMap<>(); + + @BeforeClass + public static void setup() { + // Initialize servers + for (int i = 0; i < NUM_SERVERS; i++) { + servers[i] = ServerName.valueOf("server" + i, i, System.currentTimeMillis()); + } + + // Create regions + List allRegions = new ArrayList<>(); + for (int i = 0; i < NUM_REGIONS; i++) { + TableName tableName; + if (i < 3) { + tableName = META_TABLE_NAME; + } else if (i < 30) { + tableName = SYSTEM_TABLE_NAME; + } else { + tableName = NON_ISOLATED_TABLE_NAME; + } + byte[] startKey = new byte[1]; + startKey[0] = (byte) i; + byte[] endKey = new byte[1]; + endKey[0] = (byte) (i + 1); + + RegionInfo regionInfo = + RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).setEndKey(endKey).build(); + allRegions.add(regionInfo); + } + + // Assign all regions to the first server + serverToRegions.put(servers[0], new ArrayList<>(allRegions)); + for (int i = 1; i < NUM_SERVERS; i++) { + serverToRegions.put(servers[i], new ArrayList<>()); + } + } + + @Test + public void testMultiTableIsolation() { + Configuration conf = new Configuration(false); + conf.setBoolean(BalancerConditionals.ISOLATE_META_TABLE_KEY, true); + conf.setBoolean(BalancerConditionals.ISOLATE_SYSTEM_TABLES_KEY, true); + runBalancerToExhaustion(conf, serverToRegions, + Set.of(this::isMetaTableIsolated, this::isSystemTableIsolated, this::isSystemTableColocated), + 1.0f); + LOG.info("Meta table and system table regions are successfully isolated."); + } + + /** + * Validates whether all meta table regions are isolated. + */ + private boolean isMetaTableIsolated(BalancerClusterState cluster) { + return isTableIsolated(cluster, META_TABLE_NAME, "Meta"); + } + + /** + * Validates whether all system table regions are isolated. + */ + private boolean isSystemTableIsolated(BalancerClusterState cluster) { + return isTableIsolated(cluster, SYSTEM_TABLE_NAME, "System"); + } + + /** + * Validates whether all system tables regions are colocated. We probably don't want 10 primary + * system table regions, for example, to take up 10 isolated servers. This is enforced by the + * balancer's cost functions. + */ + private boolean isSystemTableColocated(BalancerClusterState cluster) { + return isTableColocated(cluster, SYSTEM_TABLE_NAME, "System", 1) + && isTableColocated(cluster, TableName.META_TABLE_NAME, "Meta", 1); + } +} diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMultiTableIsolationReplicaDistribution.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMultiTableIsolationReplicaDistribution.java new file mode 100644 index 000000000000..b2527b7dd7ee --- /dev/null +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMultiTableIsolationReplicaDistribution.java @@ -0,0 +1,142 @@ +/* + * 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.isTableColocated; +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 TestLargeClusterBalancingMultiTableIsolationReplicaDistribution { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule + .forClass(TestLargeClusterBalancingMultiTableIsolationReplicaDistribution.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestLargeClusterBalancingMultiTableIsolationReplicaDistribution.class); + + private static final TableName META_TABLE_NAME = TableName.valueOf("hbase:meta"); + private static final TableName SYSTEM_TABLE_NAME = TableName.valueOf("hbase:system"); + private static final TableName NON_ISOLATED_TABLE_NAME = TableName.valueOf("userTable"); + + private static final int NUM_SERVERS = 100; + private static final int NUM_REGIONS = 4_000; + private static final int NUM_REPLICAS = 3; + + private static final ServerName[] servers = new ServerName[NUM_SERVERS]; + private static final Map> serverToRegions = new HashMap<>(); + + @BeforeClass + public static void setup() { + // Initialize servers + for (int i = 0; i < NUM_SERVERS; i++) { + servers[i] = ServerName.valueOf("server" + i, i, System.currentTimeMillis()); + serverToRegions.put(servers[i], new ArrayList<>()); + } + + // Create primary regions and their replicas + List allRegions = new ArrayList<>(); + for (int i = 0; i < NUM_REGIONS; i++) { + TableName tableName; + if (i < 1) { + tableName = META_TABLE_NAME; + } else if (i < 10) { + tableName = SYSTEM_TABLE_NAME; + } else { + tableName = NON_ISOLATED_TABLE_NAME; + } + + // Define startKey and endKey for the region + byte[] startKey = new byte[1]; + startKey[0] = (byte) i; + byte[] endKey = new byte[1]; + endKey[0] = (byte) (i + 1); + + // Create 3 replicas for each primary region + for (int replicaId = 0; replicaId < NUM_REPLICAS; replicaId++) { + RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey) + .setEndKey(endKey).setReplicaId(replicaId).build(); + allRegions.add(regionInfo); + } + } + + // Assign all regions to one server + for (RegionInfo regionInfo : allRegions) { + serverToRegions.get(servers[0]).add(regionInfo); + } + } + + @Test + public void testMultiTableIsolationReplicaDistribution() { + Configuration conf = new Configuration(false); + conf.setBoolean(BalancerConditionals.ISOLATE_META_TABLE_KEY, true); + conf.setBoolean(BalancerConditionals.ISOLATE_SYSTEM_TABLES_KEY, true); + conf.setBoolean(BalancerConditionals.DISTRIBUTE_REPLICAS_KEY, true); + conf.setBoolean(DistributeReplicasConditional.TEST_MODE_ENABLED_KEY, true); + + runBalancerToExhaustion(conf, serverToRegions, + Set.of(this::isMetaTableIsolated, this::isSystemTableIsolated, + CandidateGeneratorTestUtil::areAllReplicasDistributed, this::isSystemTableColocated), + 0.1f); + LOG.info("Meta table, system table, and replicas are successfully isolated."); + } + + /** + * Validates whether all meta table regions are isolated. + */ + private boolean isMetaTableIsolated(BalancerClusterState cluster) { + return isTableIsolated(cluster, META_TABLE_NAME, "Meta"); + } + + /** + * Validates whether all system table regions are isolated. + */ + private boolean isSystemTableIsolated(BalancerClusterState cluster) { + return isTableIsolated(cluster, SYSTEM_TABLE_NAME, "System"); + } + + /** + * Validates whether all system tables regions are colocated. We probably don't want 10 primary + * system table regions, for example, to take up 10 isolated servers. This is enforced by the + * balancer's cost functions. + */ + private boolean isSystemTableColocated(BalancerClusterState cluster) { + return isTableColocated(cluster, SYSTEM_TABLE_NAME, "System", NUM_REPLICAS) + && isTableColocated(cluster, TableName.META_TABLE_NAME, "Meta", NUM_REPLICAS); + } +} diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingSystemTableIsolation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingSystemTableIsolation.java new file mode 100644 index 000000000000..4b501588a105 --- /dev/null +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingSystemTableIsolation.java @@ -0,0 +1,103 @@ +/* + * 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 TestLargeClusterBalancingSystemTableIsolation { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLargeClusterBalancingSystemTableIsolation.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestLargeClusterBalancingSystemTableIsolation.class); + + private static final TableName SYSTEM_TABLE_NAME = TableName.valueOf("hbase:system"); + private static final TableName NON_SYSTEM_TABLE_NAME = TableName.valueOf("userTable"); + + private static final int NUM_SERVERS = 100; + private static final int NUM_REGIONS = 10_000; + + private static final ServerName[] servers = new ServerName[NUM_SERVERS]; + private static final Map> serverToRegions = new HashMap<>(); + + @BeforeClass + public static void setup() { + // Initialize servers + for (int i = 0; i < NUM_SERVERS; i++) { + servers[i] = ServerName.valueOf("server" + i, i, System.currentTimeMillis()); + } + + // Create regions + List allRegions = new ArrayList<>(); + for (int i = 0; i < NUM_REGIONS; i++) { + TableName tableName = i < 10 ? SYSTEM_TABLE_NAME : NON_SYSTEM_TABLE_NAME; + byte[] startKey = new byte[1]; + startKey[0] = (byte) i; + byte[] endKey = new byte[1]; + endKey[0] = (byte) (i + 1); + + RegionInfo regionInfo = + RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).setEndKey(endKey).build(); + allRegions.add(regionInfo); + } + + // Assign all regions to the first server + serverToRegions.put(servers[0], new ArrayList<>(allRegions)); + for (int i = 1; i < NUM_SERVERS; i++) { + serverToRegions.put(servers[i], new ArrayList<>()); + } + } + + @Test + public void testSystemTableIsolation() { + Configuration conf = new Configuration(false); + conf.setBoolean(BalancerConditionals.ISOLATE_SYSTEM_TABLES_KEY, true); + runBalancerToExhaustion(conf, serverToRegions, Set.of(this::isSystemTableIsolated), 0.1f); + } + + /** + * Validates whether all system table regions are isolated. + */ + private boolean isSystemTableIsolated(BalancerClusterState cluster) { + return isTableIsolated(cluster, SYSTEM_TABLE_NAME, "System"); + } + +} diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java index cc16cfe2ec83..d20d003b5ff3 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java @@ -488,7 +488,7 @@ public void testCostAfterUndoAction() { loadBalancer.initCosts(cluster); for (int i = 0; i != runs; ++i) { final double expectedCost = loadBalancer.computeCost(cluster, Double.MAX_VALUE); - BalanceAction action = loadBalancer.nextAction(cluster); + BalanceAction action = loadBalancer.nextAction(cluster).getSecond(); cluster.doAction(action); loadBalancer.updateCostsAndWeightsWithAction(cluster, action); BalanceAction undoAction = action.undoAction(); diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java index 2a7b8afccc38..aca3cbca9e8a 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java @@ -51,6 +51,7 @@ public class TestStochasticLoadBalancerBalanceCluster extends StochasticBalancer */ @Test public void testBalanceCluster() throws Exception { + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); loadBalancer.onConfigurationChange(conf); for (int[] mockCluster : clusterStateMocks) { Map> servers = mockClusterServers(mockCluster); diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerHeterogeneousCost.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerHeterogeneousCost.java index 8691cff733f5..a562ecf4845e 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerHeterogeneousCost.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerHeterogeneousCost.java @@ -306,8 +306,12 @@ static class StochasticLoadTestBalancer extends StochasticLoadBalancer { private FairRandomCandidateGenerator fairRandomCandidateGenerator = new FairRandomCandidateGenerator(); + StochasticLoadTestBalancer() { + super(new DummyMetricsStochasticBalancer()); + } + @Override - protected CandidateGenerator getRandomGenerator() { + protected CandidateGenerator getRandomGenerator(BalancerClusterState cluster) { return fairRandomCandidateGenerator; } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java index 620d610288f7..06e2e0522ccf 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java @@ -38,7 +38,7 @@ public void testLargeCluster() { int numRegionsPerServer = 80; // all servers except one int numTables = 100; int replication = 1; - conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 6 * 60 * 1000); + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 30_000); loadBalancer.onConfigurationChange(conf); testWithClusterWithIteration(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerMidCluster.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerMidCluster.java index 80adea40fd77..5137f52822ec 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerMidCluster.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerMidCluster.java @@ -38,7 +38,10 @@ public void testMidCluster() { int numRegionsPerServer = 60; // all servers except one int replication = 1; int numTables = 40; - testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); + loadBalancer.onConfigurationChange(conf); + testWithClusterWithIteration(numNodes, numRegions, numRegionsPerServer, replication, numTables, + true, true); } @Test @@ -50,7 +53,9 @@ public void testMidCluster2() { int numTables = 400; // num large num regions means may not always get to best balance with one run boolean assertFullyBalanced = false; - testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); + loadBalancer.onConfigurationChange(conf); + testWithClusterWithIteration(numNodes, numRegions, numRegionsPerServer, replication, numTables, assertFullyBalanced, false); } @@ -61,7 +66,10 @@ public void testMidCluster3() { int numRegionsPerServer = 9; // all servers except one int replication = 1; int numTables = 110; - testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); + loadBalancer.onConfigurationChange(conf); + testWithClusterWithIteration(numNodes, numRegions, numRegionsPerServer, replication, numTables, + true, true); // TODO(eclark): Make sure that the tables are well distributed. } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java index bd437425f213..729e4056a938 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -137,29 +138,60 @@ public void testReplicaCostForReplicas() { @Test public void testNeedsBalanceForColocatedReplicas() { - // check for the case where there are two hosts and with one rack, and where - // both the replicas are hosted on the same server + // 1) Same-server scenario: two replicas on the same server => needsBalance == true List regions = randomRegions(1); ServerName s1 = ServerName.valueOf("host1", 1000, 11111); ServerName s2 = ServerName.valueOf("host11", 1000, 11111); Map> map = new HashMap<>(); + + // Place the first region on s1 map.put(s1, regions); + // Add a replica for that region, also on s1 => same server regions.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1)); - // until the step above s1 holds two replicas of a region + // Put another (unrelated) region on s2 regions = randomRegions(1); map.put(s2, regions); + + // Since two replicas of the same region are on s1, we expect balance needed assertTrue(loadBalancer.needsBalance(HConstants.ENSEMBLE_TABLE_NAME, new BalancerClusterState(map, null, null, null))); - // check for the case where there are two hosts on the same rack and there are two racks - // and both the replicas are on the same rack + + // 2) Same-rack scenario: two replicas on different servers but the same rack => needsBalance == + // true map.clear(); regions = randomRegions(1); - List regionsOnS2 = new ArrayList<>(1); - regionsOnS2.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1)); + // Place the region on s1 map.put(s1, regions); + // Place its replica on s2 (which ForTestRackManagerOne() puts on the same rack as s1) + List regionsOnS2 = new ArrayList<>(); + RegionInfo replicaR1 = RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1); + regionsOnS2.add(replicaR1); map.put(s2, regionsOnS2); - // add another server so that the cluster has some host on another rack - map.put(ServerName.valueOf("host2", 1000, 11111), randomRegions(1)); + + // Add a third server on a different rack + ServerName s3 = ServerName.valueOf("host2", 1000, 11111); + map.put(s3, randomRegions(1)); + + // Now that needsBalance checks for same-rack co-location, we expect true + assertTrue(loadBalancer.needsBalance(HConstants.ENSEMBLE_TABLE_NAME, + new BalancerClusterState(map, null, null, new ForTestRackManagerOne()))); + + // 3) No co-location scenario: ensure needsBalance == false + // Clear the map and place each region/replica on a different rack/server + map.clear(); + // Make a new region + List distinctRegions = randomRegions(1); + RegionInfo replicaR2 = RegionReplicaUtil.getRegionInfoForReplica(distinctRegions.get(0), 1); + + // Put the original region on s1 + map.put(s1, new ArrayList<>(Collections.singletonList(distinctRegions.get(0)))); + // Put the replica on s3 (which is on a different rack than s1 and s2) + map.put(s3, new ArrayList<>(Collections.singletonList(replicaR2))); + // Put some random region on s2, just to fill out the cluster + map.put(s2, randomRegions(1)); + + // Because each replica is now on a distinct server *and* a distinct rack, + // we expect that the balancer does NOT see co-location => false assertFalse(loadBalancer.needsBalance(HConstants.ENSEMBLE_TABLE_NAME, new BalancerClusterState(map, null, null, new ForTestRackManagerOne()))); } @@ -171,7 +203,10 @@ public void testRegionReplicasOnSmallCluster() { int replication = 3; // 3 replicas per region int numRegionsPerServer = 80; // all regions are mostly balanced int numTables = 10; - testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); + loadBalancer.onConfigurationChange(conf); + testWithClusterWithIteration(numNodes, numRegions, numRegionsPerServer, replication, numTables, + true, true); } private static class ForTestRackManagerOne extends RackManager { diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java index a58b8e162968..9e8d1c01f3da 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java @@ -35,13 +35,14 @@ public class TestStochasticLoadBalancerRegionReplicaHighReplication @Test public void testRegionReplicasOnMidClusterHighReplication() { conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 4000000L); - conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); loadBalancer.onConfigurationChange(conf); int numNodes = 40; int numRegions = 6 * numNodes; int replication = 40; // 40 replicas per region, one for each server int numRegionsPerServer = 5; int numTables = 10; - testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, false, true); + testWithClusterWithIteration(numNodes, numRegions, numRegionsPerServer, replication, numTables, + false, true); } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaLargeCluster.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaLargeCluster.java index af010033ae55..cd985549df69 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaLargeCluster.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaLargeCluster.java @@ -39,12 +39,16 @@ public void testRegionReplicasOnLargeCluster() { // ignore these two cost functions to allow us to make any move that helps other functions. conf.setFloat("hbase.master.balancer.stochastic.moveCost", 0f); conf.setFloat("hbase.master.balancer.stochastic.tableSkewCost", 0f); + conf.setBoolean("hbase.master.balancer.stochastic.runMaxSteps", true); + conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 100000000L); + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 30_000); loadBalancer.onConfigurationChange(conf); int numNodes = 1000; int numRegions = 20 * numNodes; // 20 * replication regions per RS int numRegionsPerServer = 19; // all servers except one int numTables = 100; int replication = 3; - testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); + testWithClusterWithIteration(numNodes, numRegions, numRegionsPerServer, replication, numTables, + true, true); } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaMidCluster.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaMidCluster.java index e668b20c1cb3..eac15f046b2e 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaMidCluster.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaMidCluster.java @@ -33,11 +33,16 @@ public class TestStochasticLoadBalancerRegionReplicaMidCluster extends Stochasti @Test public void testRegionReplicasOnMidCluster() { + conf.setBoolean("hbase.master.balancer.stochastic.runMaxSteps", true); + conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 100000000L); + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); + loadBalancer.onConfigurationChange(conf); int numNodes = 200; int numRegions = 40 * 200; int replication = 3; // 3 replicas per region int numRegionsPerServer = 30; // all regions are mostly balanced int numTables = 10; - testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); + testWithClusterWithIteration(numNodes, numRegions, numRegionsPerServer, replication, numTables, + true, true); } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNodes.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNodes.java index 6ee9682e8d66..c639c5e2aea3 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNodes.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNodes.java @@ -34,13 +34,14 @@ public class TestStochasticLoadBalancerRegionReplicaReplicationGreaterThanNumNod @Test public void testRegionReplicationOnMidClusterReplicationGreaterThanNumNodes() { - conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); loadBalancer.onConfigurationChange(conf); int numNodes = 40; int numRegions = 6 * 50; int replication = 50; // 50 replicas per region, more than numNodes int numRegionsPerServer = 6; int numTables = 10; - testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, false); + testWithClusterWithIteration(numNodes, numRegions, numRegionsPerServer, replication, numTables, + true, false); } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaSameHosts.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaSameHosts.java index c8e7a83c9bd4..f8172e2273db 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaSameHosts.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaSameHosts.java @@ -40,7 +40,7 @@ public class TestStochasticLoadBalancerRegionReplicaSameHosts extends Stochastic @Test public void testRegionReplicationOnMidClusterSameHosts() { conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L); - conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); loadBalancer.onConfigurationChange(conf); int numHosts = 30; int numRegions = 30 * 30; @@ -62,6 +62,6 @@ public void testRegionReplicationOnMidClusterSameHosts() { } } - testWithCluster(newServerMap, null, true, true); + testWithClusterWithIteration(newServerMap, null, true, true); } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaWithRacks.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaWithRacks.java index ad60aadc1c17..d97e544e137e 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaWithRacks.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaWithRacks.java @@ -61,7 +61,7 @@ public String getRack(ServerName server) { public void testRegionReplicationOnMidClusterWithRacks() { conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 100000000L); conf.setBoolean("hbase.master.balancer.stochastic.runMaxSteps", true); - conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); // 10 sec loadBalancer.onConfigurationChange(conf); int numNodes = 5; int numRegions = numNodes * 1; @@ -79,7 +79,7 @@ public void testRegionReplicationOnMidClusterWithRacks() { public void testRegionReplicationOnLargeClusterWithRacks() { conf.setBoolean("hbase.master.balancer.stochastic.runMaxSteps", true); conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 100000000L); - conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 10 sec + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10 * 1000); // 10 sec loadBalancer.onConfigurationChange(conf); int numNodes = 100; int numRegions = numNodes * 30; diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java index 831c9e932727..fe72bed7558e 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java @@ -38,6 +38,8 @@ public void testSmallCluster() { int numRegionsPerServer = 40; // all servers except one int replication = 1; int numTables = 10; + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); + loadBalancer.onConfigurationChange(conf); testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); } @@ -48,6 +50,8 @@ public void testSmallCluster2() { int numRegionsPerServer = 40; // all servers except one int replication = 1; int numTables = 10; + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); + loadBalancer.onConfigurationChange(conf); testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); } @@ -58,6 +62,8 @@ public void testSmallCluster3() { int numRegionsPerServer = 1; // all servers except one int replication = 1; int numTables = 10; + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 10_000); + loadBalancer.onConfigurationChange(conf); // fails because of max moves testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, false, false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java new file mode 100644 index 000000000000..8a7169b09309 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java @@ -0,0 +1,221 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; + +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +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.quotas.QuotaUtil; +import org.apache.hadoop.hbase.util.Bytes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; + +public final class BalancerConditionalsTestUtil { + + private static final Logger LOG = LoggerFactory.getLogger(BalancerConditionalsTestUtil.class); + + private BalancerConditionalsTestUtil() { + } + + static byte[][] generateSplits(int numRegions) { + byte[][] splitKeys = new byte[numRegions - 1][]; + for (int i = 0; i < numRegions - 1; i++) { + splitKeys[i] = + Bytes.toBytes(String.format("%09d", (i + 1) * (Integer.MAX_VALUE / numRegions))); + } + return splitKeys; + } + + static void printRegionLocations(Connection connection) throws IOException { + Admin admin = connection.getAdmin(); + + // Get all table names in the cluster + Set tableNames = admin.listTableDescriptors(true).stream() + .map(TableDescriptor::getTableName).collect(Collectors.toSet()); + + // Group regions by server + Map>> serverToRegions = + admin.getClusterMetrics().getLiveServerMetrics().keySet().stream() + .collect(Collectors.toMap(server -> server, server -> { + try { + return listRegionsByTable(connection, server, tableNames); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + + // Pretty print region locations + StringBuilder regionLocationOutput = new StringBuilder(); + regionLocationOutput.append("Pretty printing region locations...\n"); + serverToRegions.forEach((server, tableRegions) -> { + regionLocationOutput.append("Server: " + server.getServerName() + "\n"); + tableRegions.forEach((table, regions) -> { + if (regions.isEmpty()) { + return; + } + regionLocationOutput.append(" Table: " + table.getNameAsString() + "\n"); + regions.forEach(region -> regionLocationOutput + .append(String.format(" Region: %s, start: %s, end: %s, replica: %s\n", + region.getEncodedName(), Bytes.toString(region.getStartKey()), + Bytes.toString(region.getEndKey()), region.getReplicaId()))); + }); + }); + LOG.info(regionLocationOutput.toString()); + } + + private static Map> listRegionsByTable(Connection connection, + ServerName server, Set tableNames) throws IOException { + Admin admin = connection.getAdmin(); + + // Find regions for each table + return tableNames.stream().collect(Collectors.toMap(tableName -> tableName, tableName -> { + List allRegions = null; + try { + allRegions = admin.getRegions(server); + } catch (IOException e) { + throw new RuntimeException(e); + } + return allRegions.stream().filter(region -> region.getTable().equals(tableName)) + .collect(Collectors.toList()); + })); + } + + static void validateReplicaDistribution(Connection connection, TableName tableName, + boolean shouldBeDistributed) { + Map> serverToRegions = null; + try { + serverToRegions = connection.getRegionLocator(tableName).getAllRegionLocations().stream() + .collect(Collectors.groupingBy(location -> location.getServerName(), + Collectors.mapping(location -> location.getRegion(), Collectors.toList()))); + } catch (IOException e) { + throw new RuntimeException(e); + } + + if (shouldBeDistributed) { + // Ensure no server hosts more than one replica of any region + for (Map.Entry> serverAndRegions : serverToRegions.entrySet()) { + List regionInfos = serverAndRegions.getValue(); + Set startKeys = new HashSet<>(); + for (RegionInfo regionInfo : regionInfos) { + // each region should have a distinct start key + assertFalse( + "Each region should have its own start key, " + + "demonstrating it is not a replica of any others on this host", + startKeys.contains(regionInfo.getStartKey())); + startKeys.add(regionInfo.getStartKey()); + } + } + } else { + // Ensure all replicas are on the same server + assertEquals("All regions should share one server", 1, serverToRegions.size()); + } + } + + static void validateRegionLocations(Map> tableToServers, + TableName productTableName, boolean shouldBeBalanced) { + ServerName metaServer = + tableToServers.get(TableName.META_TABLE_NAME).stream().findFirst().orElseThrow(); + ServerName quotaServer = + tableToServers.get(QuotaUtil.QUOTA_TABLE_NAME).stream().findFirst().orElseThrow(); + Set productServers = tableToServers.get(productTableName); + + if (shouldBeBalanced) { + for (ServerName server : productServers) { + assertNotEquals("Meta table and product table should not share servers", server, + metaServer); + assertNotEquals("Quota table and product table should not share servers", server, + quotaServer); + } + assertNotEquals("The meta server and quotas server should be different", metaServer, + quotaServer); + } else { + for (ServerName server : productServers) { + assertEquals("Meta table and product table must share servers", server, metaServer); + assertEquals("Quota table and product table must share servers", server, quotaServer); + } + assertEquals("The meta server and quotas server must be the same", metaServer, quotaServer); + } + } + + static Map> getTableToServers(Connection connection, + Set tableNames) { + return tableNames.stream().collect(Collectors.toMap(t -> t, t -> { + try { + return connection.getRegionLocator(t).getAllRegionLocations().stream() + .map(HRegionLocation::getServerName).collect(Collectors.toSet()); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + } + + @FunctionalInterface + interface AssertionRunnable { + void run() throws AssertionError; + } + + static void validateAssertionsWithRetries(HBaseTestingUtil testUtil, boolean runBalancerOnFailure, + AssertionRunnable assertion) { + validateAssertionsWithRetries(testUtil, runBalancerOnFailure, ImmutableSet.of(assertion)); + } + + static void validateAssertionsWithRetries(HBaseTestingUtil testUtil, boolean runBalancerOnFailure, + Set assertions) { + int maxAttempts = 50; + for (int i = 0; i < maxAttempts; i++) { + try { + for (AssertionRunnable assertion : assertions) { + assertion.run(); + } + } catch (AssertionError e) { + if (i == maxAttempts - 1) { + throw e; + } + try { + LOG.warn("Failed to validate region locations. Will retry", e); + Thread.sleep(1000); + BalancerConditionalsTestUtil.printRegionLocations(testUtil.getConnection()); + if (runBalancerOnFailure) { + testUtil.getAdmin().balance(); + } + Thread.sleep(1000); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + } + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadOnlyFavoredStochasticBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadOnlyFavoredStochasticBalancer.java index ba6ee4af843f..d658f7cfa167 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadOnlyFavoredStochasticBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadOnlyFavoredStochasticBalancer.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hbase.master.balancer; -import java.util.ArrayList; -import java.util.List; +import java.util.HashMap; +import java.util.Map; /** * Used for FavoredNode unit tests @@ -26,9 +26,10 @@ public class LoadOnlyFavoredStochasticBalancer extends FavoredStochasticBalancer { @Override - protected List createCandidateGenerators() { - List fnPickers = new ArrayList<>(1); - fnPickers.add(new FavoredNodeLoadPicker()); + protected Map, CandidateGenerator> + createCandidateGenerators() { + Map, CandidateGenerator> fnPickers = new HashMap<>(1); + fnPickers.put(FavoredNodeLoadPicker.class, new FavoredNodeLoadPicker()); return fnPickers; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestReplicaDistributionBalancerConditional.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestReplicaDistributionBalancerConditional.java new file mode 100644 index 000000000000..e60ed8f5957b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestReplicaDistributionBalancerConditional.java @@ -0,0 +1,120 @@ +/* + * 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.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.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_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); + } + + @After + public void tearDown() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testReplicaDistribution() throws Exception { + Connection connection = TEST_UTIL.getConnection(); + Admin admin = connection.getAdmin(); + + // Create a "replicated_table" with region replicas + TableName replicatedTableName = TableName.valueOf("replicated_table"); + TableDescriptor replicatedTableDescriptor = + TableDescriptorBuilder.newBuilder(replicatedTableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("0")).build()) + .setRegionReplication(REPLICAS).build(); + admin.createTable(replicatedTableDescriptor, + BalancerConditionalsTestUtil.generateSplits(REGIONS_PER_SERVER * NUM_SERVERS)); + + // Pause the balancer + admin.balancerSwitch(false, true); + + // Collect all region replicas and place them on one RegionServer + List allRegions = admin.getRegions(replicatedTableName); + String targetServer = + TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName().getServerName(); + + for (RegionInfo region : allRegions) { + admin.move(region.getEncodedNameAsBytes(), Bytes.toBytes(targetServer)); + } + + BalancerConditionalsTestUtil.printRegionLocations(TEST_UTIL.getConnection()); + validateAssertionsWithRetries(TEST_UTIL, false, () -> BalancerConditionalsTestUtil + .validateReplicaDistribution(connection, replicatedTableName, false)); + + // Unpause the balancer and trigger balancing + admin.balancerSwitch(true, true); + admin.balance(); + + validateAssertionsWithRetries(TEST_UTIL, true, () -> BalancerConditionalsTestUtil + .validateReplicaDistribution(connection, replicatedTableName, true)); + BalancerConditionalsTestUtil.printRegionLocations(TEST_UTIL.getConnection()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestSystemTableIsolationBalancerConditionals.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestSystemTableIsolationBalancerConditionals.java new file mode 100644 index 000000000000..2e2cc90d11ef --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestSystemTableIsolationBalancerConditionals.java @@ -0,0 +1,127 @@ +/* + * 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.BalancerConditionalsTestUtil.getTableToServers; +import static org.apache.hadoop.hbase.master.balancer.BalancerConditionalsTestUtil.validateAssertionsWithRetries; +import static org.apache.hadoop.hbase.master.balancer.BalancerConditionalsTestUtil.validateRegionLocations; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Set; +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.quotas.QuotaUtil; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +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; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; + +@Category(LargeTests.class) +public class TestSystemTableIsolationBalancerConditionals { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSystemTableIsolationBalancerConditionals.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestSystemTableIsolationBalancerConditionals.class); + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + + // One for product table, one for meta, one for other system tables, and one extra + private static final int NUM_SERVERS = 3; + private static final int PRODUCT_TABLE_REGIONS_PER_SERVER = 5; + + @Before + public void setUp() throws Exception { + TEST_UTIL.getConfiguration().setBoolean(BalancerConditionals.ISOLATE_SYSTEM_TABLES_KEY, true); + TEST_UTIL.getConfiguration().setBoolean(BalancerConditionals.ISOLATE_META_TABLE_KEY, true); + TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true); + TEST_UTIL.getConfiguration().setLong(HConstants.HBASE_BALANCER_PERIOD, 1000L); + TEST_UTIL.getConfiguration().setBoolean("hbase.master.balancer.stochastic.runMaxSteps", true); + + TEST_UTIL.startMiniCluster(NUM_SERVERS); + } + + @After + public void tearDown() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testTableIsolation() throws Exception { + Connection connection = TEST_UTIL.getConnection(); + Admin admin = connection.getAdmin(); + + // Create "product" table with 3 regions + TableName productTableName = TableName.valueOf("product"); + TableDescriptor productTableDescriptor = TableDescriptorBuilder.newBuilder(productTableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("0")).build()) + .build(); + admin.createTable(productTableDescriptor, + BalancerConditionalsTestUtil.generateSplits(PRODUCT_TABLE_REGIONS_PER_SERVER * NUM_SERVERS)); + + Set tablesToBeSeparated = ImmutableSet. builder() + .add(TableName.META_TABLE_NAME).add(QuotaUtil.QUOTA_TABLE_NAME).add(productTableName).build(); + + // Pause the balancer + admin.balancerSwitch(false, true); + + // Move all regions (product, meta, and quotas) to one RegionServer + List allRegions = tablesToBeSeparated.stream().map(t -> { + try { + return admin.getRegions(t); + } catch (IOException e) { + throw new RuntimeException(e); + } + }).flatMap(Collection::stream).toList(); + String targetServer = + TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName().getServerName(); + for (RegionInfo region : allRegions) { + admin.move(region.getEncodedNameAsBytes(), Bytes.toBytes(targetServer)); + } + + validateAssertionsWithRetries(TEST_UTIL, false, + () -> validateRegionLocations(getTableToServers(connection, tablesToBeSeparated), + productTableName, false)); + + // Unpause the balancer and run it + admin.balancerSwitch(true, true); + admin.balance(); + + validateAssertionsWithRetries(TEST_UTIL, true, + () -> validateRegionLocations(getTableToServers(connection, tablesToBeSeparated), + productTableName, true)); + } +}