From 487135d3fbd5b5684a72de5f0909cbd4eb7751b9 Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Mon, 31 Jul 2023 22:03:28 +0530 Subject: [PATCH 1/5] HBASE-27999 Implement cache prefetch aware load balancer --- .../master/balancer/BalancerClusterState.java | 155 +++++- .../master/balancer/BalancerRegionLoad.java | 12 + .../master/balancer/BaseLoadBalancer.java | 3 +- .../balancer/PrefetchAwareLoadBalancer.java | 468 ++++++++++++++++++ .../balancer/StochasticLoadBalancer.java | 52 +- .../master/balancer/BalancerTestBase.java | 14 + .../balancer/TestStochasticLoadBalancer.java | 4 + .../org/apache/hadoop/hbase/HConstants.java | 5 + .../hbase/io/hfile/bucket/BucketCache.java | 1 + .../apache/hadoop/hbase/HBaseTestingUtil.java | 124 +++++ .../TestBlockEvictionOnRegionMovement.java | 1 + .../hbase/io/hfile/TestPrefetchRSClose.java | 1 + .../TestPrefetchAwareLoadBalancer.java | 403 +++++++++++++++ ...refetchAwareLoadBalancerCostFunctions.java | 319 ++++++++++++ 14 files changed, 1540 insertions(+), 22 deletions(-) create mode 100644 hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancer.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancerCostFunctions.java 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 a7ae8b4d1a5a..1078ade65545 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 @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.master.RackManager; 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; @@ -114,6 +115,12 @@ class BalancerClusterState { private float[][] rackLocalities; // Maps localityType -> region -> [server|rack]Index with highest locality private int[][] regionsToMostLocalEntities; + // Maps region -> serverIndex -> prefetchRatio of a region on a server + private Map, Float> regionIndexServerIndexPrefetchRatio; + // Maps regionIndex -> serverIndex with best prefetch ratio + private int[] regionServerIndexWithBestPrefetchRatio; + // Maps regionName -> oldServerName -> oldPrefetchRatio + Map> oldRegionServerPrefetchRatio; static class DefaultRackManager extends RackManager { @Override @@ -125,13 +132,20 @@ public String getRack(ServerName server) { BalancerClusterState(Map> clusterState, Map> loads, RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager) { - this(null, clusterState, loads, regionFinder, rackManager); + this(null, clusterState, loads, regionFinder, rackManager, null); + } + + protected BalancerClusterState(Map> clusterState, + Map> loads, RegionHDFSBlockLocationFinder regionFinder, + RackManager rackManager, Map> oldRegionServerPrefetchRatio) { + this(null, clusterState, loads, regionFinder, rackManager, oldRegionServerPrefetchRatio); } @SuppressWarnings("unchecked") BalancerClusterState(Collection unassignedRegions, Map> clusterState, Map> loads, - RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager) { + RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager, + Map> oldRegionServerPrefetchRatio) { if (unassignedRegions == null) { unassignedRegions = Collections.emptyList(); } @@ -145,6 +159,8 @@ public String getRack(ServerName server) { tables = new ArrayList<>(); this.rackManager = rackManager != null ? rackManager : new DefaultRackManager(); + this.oldRegionServerPrefetchRatio = oldRegionServerPrefetchRatio; + numRegions = 0; List> serversPerHostList = new ArrayList<>(); @@ -541,6 +557,141 @@ private void computeCachedLocalities() { } + /** + * Returns the size of hFiles from the most recent RegionLoad for region + */ + public int getTotalRegionHFileSizeMB(int region) { + Deque load = regionLoads[region]; + if (load == null) { + // This means, that the region has no actual data on disk + return 0; + } + return regionLoads[region].getLast().getRegionSizeMB(); + } + + /** + * Returns the weighted prefetch ratio of a region on the given region server + */ + public float getOrComputeWeightedPrefetchRatio(int region, int server) { + return getTotalRegionHFileSizeMB(region) * getOrComputeRegionPrefetchRatio(region, server); + } + + /** + * Returns the amount by which a region is prefetched on a given region server. If the region is + * not currently hosted on the given region server, then find out if it was previously hosted + * there and return the old prefetch ratio. + */ + protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) { + float prefetchRatio = 0.0f; + + // Get the current prefetch ratio if the region is hosted on the server regionServerIndex + for (int regionIndex : regionsPerServer[regionServerIndex]) { + if (region != regionIndex) { + continue; + } + + Deque regionLoadList = regionLoads[regionIndex]; + + // The region is currently hosted on this region server. Get the prefetch ratio for this + // region on this server + prefetchRatio = + regionLoadList == null ? 0.0f : regionLoadList.getLast().getCurrentRegionPrefetchRatio(); + + return prefetchRatio; + } + + // Region is not currently hosted on this server. Check if the region was prefetched on this + // server earlier. This can happen when the server was shutdown and the cache was persisted. + // Search using the region name and server name and not the index id and server id as these ids + // may change when a server is marked as dead or a new server is added. + String regionEncodedName = regions[region].getEncodedName(); + ServerName serverName = servers[regionServerIndex]; + if ( + oldRegionServerPrefetchRatio != null + && oldRegionServerPrefetchRatio.containsKey(regionEncodedName) + ) { + Pair serverPrefetchRatio = + oldRegionServerPrefetchRatio.get(regionEncodedName); + if (ServerName.isSameAddress(serverPrefetchRatio.getFirst(), serverName)) { + prefetchRatio = serverPrefetchRatio.getSecond(); + if (LOG.isDebugEnabled()) { + LOG.debug("Old prefetch ratio found for region {} on server {}: {}", regionEncodedName, + serverName, prefetchRatio); + } + } + } + return prefetchRatio; + } + + /** + * Populate the maps containing information about how much a region is prefetched on a region + * server. + */ + private void computeRegionServerPrefetchRatio() { + regionIndexServerIndexPrefetchRatio = new HashMap<>(); + regionServerIndexWithBestPrefetchRatio = new int[numRegions]; + + for (int region = 0; region < numRegions; region++) { + float bestPrefetchRatio = 0.0f; + int serverWithBestPrefetchRatio = 0; + for (int server = 0; server < numServers; server++) { + float prefetchRatio = getRegionServerPrefetchRatio(region, server); + if (prefetchRatio > 0.0f || server == regionIndexToServerIndex[region]) { + // A region with prefetch ratio 0 on a server means nothing. Hence, just make a note of + // prefetch only if the prefetch ratio is greater than 0. + Pair regionServerPair = new Pair<>(region, server); + regionIndexServerIndexPrefetchRatio.put(regionServerPair, prefetchRatio); + } + if (prefetchRatio > bestPrefetchRatio) { + serverWithBestPrefetchRatio = server; + // If the server currently hosting the region has equal prefetch ratio to a historical + // server, consider the current server to keep hosting the region + bestPrefetchRatio = prefetchRatio; + } else + if (prefetchRatio == bestPrefetchRatio && server == regionIndexToServerIndex[region]) { + // If two servers have same prefetch ratio, then the server currently hosting the region + // should retain the region + serverWithBestPrefetchRatio = server; + } + } + regionServerIndexWithBestPrefetchRatio[region] = serverWithBestPrefetchRatio; + Pair regionServerPair = + new Pair<>(region, regionIndexToServerIndex[region]); + float tempPrefetchRatio = regionIndexServerIndexPrefetchRatio.get(regionServerPair); + if (tempPrefetchRatio > bestPrefetchRatio) { + LOG.warn( + "INVALID CONDITION: region {} on server {} prefetch ratio {} is greater than the " + + "best prefetch ratio {} on server {}", + regions[region].getEncodedName(), servers[regionIndexToServerIndex[region]], + tempPrefetchRatio, bestPrefetchRatio, servers[serverWithBestPrefetchRatio]); + } + } + } + + protected float getOrComputeRegionPrefetchRatio(int region, int server) { + if ( + regionServerIndexWithBestPrefetchRatio == null + || regionIndexServerIndexPrefetchRatio.isEmpty() + ) { + computeRegionServerPrefetchRatio(); + } + + Pair regionServerPair = new Pair<>(region, server); + return regionIndexServerIndexPrefetchRatio.containsKey(regionServerPair) + ? regionIndexServerIndexPrefetchRatio.get(regionServerPair) + : 0.0f; + } + + public int[] getOrComputeServerWithBestPrefetchRatio() { + if ( + regionServerIndexWithBestPrefetchRatio == null + || regionIndexServerIndexPrefetchRatio.isEmpty() + ) { + computeRegionServerPrefetchRatio(); + } + return regionServerIndexWithBestPrefetchRatio; + } + /** * Maps region index to rack index */ diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java index ffb36cb8ca1a..ad6cc1e0e9a6 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java @@ -34,6 +34,8 @@ class BalancerRegionLoad { private final long writeRequestsCount; private final int memStoreSizeMB; private final int storefileSizeMB; + private final int regionSizeMB; + private final float currentRegionPrefetchRatio; BalancerRegionLoad(RegionMetrics regionMetrics) { readRequestsCount = regionMetrics.getReadRequestCount(); @@ -41,6 +43,8 @@ class BalancerRegionLoad { writeRequestsCount = regionMetrics.getWriteRequestCount(); memStoreSizeMB = (int) regionMetrics.getMemStoreSize().get(Size.Unit.MEGABYTE); storefileSizeMB = (int) regionMetrics.getStoreFileSize().get(Size.Unit.MEGABYTE); + regionSizeMB = (int) regionMetrics.getRegionSizeMB().get(Size.Unit.MEGABYTE); + currentRegionPrefetchRatio = regionMetrics.getCurrentRegionCachedRatio(); } public long getReadRequestsCount() { @@ -62,4 +66,12 @@ public int getMemStoreSizeMB() { public int getStorefileSizeMB() { return storefileSizeMB; } + + public int getRegionSizeMB() { + return regionSizeMB; + } + + public float getCurrentRegionPrefetchRatio() { + return currentRegionPrefetchRatio; + } } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java index a4560cc595a2..54516868a0a0 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java @@ -232,7 +232,8 @@ private BalancerClusterState createCluster(List servers, clusterState.put(server, Collections.emptyList()); } } - return new BalancerClusterState(regions, clusterState, null, this.regionFinder, rackManager); + return new BalancerClusterState(regions, clusterState, null, this.regionFinder, rackManager, + null); } private List findIdleServers(List servers) { diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java new file mode 100644 index 000000000000..8de37f3158c8 --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java @@ -0,0 +1,468 @@ +/* + * 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; + +/** An implementation of the {@link org.apache.hadoop.hbase.master.LoadBalancer} that assigns regions + * based on the amount they are prefetched on a given server. A region can move across the region + * servers whenever a region server shuts down or crashes. The region server preserves the cache + * periodically and restores the cache when it is restarted. This balancer implements a mechanism + * where it maintains the amount by which a region is prefetched on a region server. During balancer + * run, a region plan is generated that takes into account this prefetch information and tries to + * move the regions so that the cache minimally impacted. + */ + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Deque; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ClusterMetrics; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.RegionMetrics; +import org.apache.hadoop.hbase.ServerMetrics; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.Size; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +public class PrefetchAwareLoadBalancer extends StochasticLoadBalancer { + private static final Logger LOG = LoggerFactory.getLogger(PrefetchAwareLoadBalancer.class); + + private Configuration configuration; + + public enum GeneratorFunctionType { + LOAD, + PREFETCH + } + + @Override + public synchronized void loadConf(Configuration configuration) { + this.configuration = configuration; + this.costFunctions = new ArrayList<>(); + super.loadConf(configuration); + } + + @Override + protected List createCandidateGenerators() { + List candidateGenerators = new ArrayList<>(2); + candidateGenerators.add(GeneratorFunctionType.LOAD.ordinal(), + new PrefetchBasedSkewnessCandidateGenerator()); + candidateGenerators.add(GeneratorFunctionType.PREFETCH.ordinal(), + new PrefetchCandidateGenerator()); + return candidateGenerators; + } + + @Override + protected List createCostFunctions(Configuration configuration) { + List costFunctions = new ArrayList<>(); + addCostFunction(costFunctions, new PrefetchAwareRegionSkewnessCostFunction(configuration)); + addCostFunction(costFunctions, new PrefetchCacheCostFunction(configuration)); + return costFunctions; + } + + private void addCostFunction(List costFunctions, CostFunction costFunction) { + if (costFunction.getMultiplier() > 0) { + costFunctions.add(costFunction); + } + } + + @Override + public synchronized void updateClusterMetrics(ClusterMetrics clusterMetrics) { + this.clusterStatus = clusterMetrics; + updateRegionLoad(); + } + + /** + * Collect the prefetch information for all the regions from all the active region servers. + */ + private synchronized void updateRegionLoad() { + loads = new HashMap<>(); + oldRegionPrefetchMap = new HashMap<>(); + Map> currentPrefetchMap = new HashMap<>(); + + // Build current region prefetch statistics + clusterStatus.getLiveServerMetrics().forEach((ServerName sn, ServerMetrics sm) -> { + // Create a map of region and the server where it is currently hosted + sm.getRegionMetrics().forEach((byte[] regionName, RegionMetrics rm) -> { + String regionEncodedName = RegionInfo.encodeRegionName(regionName); + + Deque rload = new ArrayDeque<>(); + + // Get the total size of the hFiles in this region + int regionSizeMB = (int) rm.getRegionSizeMB().get(Size.Unit.MEGABYTE); + + rload.add(new BalancerRegionLoad(rm)); + // Maintain a map of region and it's total size. This is needed to calculate the prefetch + // ratios for the regions cached on old region servers + currentPrefetchMap.put(regionEncodedName, new Pair<>(sn, regionSizeMB)); + loads.put(regionEncodedName, rload); + }); + }); + + // Build olf region prefetch statistics + clusterStatus.getLiveServerMetrics().forEach((ServerName sn, ServerMetrics sm) -> { + // Find if a region was previously hosted on a server other than the one it is currently + // hosted on. + sm.getRegionCachedInfo().forEach((String regionEncodedName, Integer prefetchSize) -> { + // If the region is fond in currentPrefetchMap, it is currently hosted on this server + if (currentPrefetchMap.containsKey(regionEncodedName)) { + ServerName currentServer = currentPrefetchMap.get(regionEncodedName).getFirst(); + if (!ServerName.isSameAddress(currentServer, sn)) { + int regionSizeMB = currentPrefetchMap.get(regionEncodedName).getSecond(); + float regionPrefetchRatio = + regionSizeMB == 0 ? 0.0f : (float) prefetchSize / regionSizeMB; + oldRegionPrefetchMap.put(regionEncodedName, new Pair<>(sn, regionPrefetchRatio)); + } + } else { + LOG.warn("Region {} not currently hosted on any server", regionEncodedName); + } + }); + }); + } + + private RegionInfo getRegionInfoByEncodedName(BalancerClusterState cluster, String regionName) { + Optional regionInfoOptional = + Arrays.stream(cluster.regions).filter((RegionInfo ri) -> { + return regionName.equals(ri.getEncodedName()); + }).findFirst(); + + if (regionInfoOptional.isPresent()) { + return regionInfoOptional.get(); + } + return null; + } + + private class PrefetchCandidateGenerator extends CandidateGenerator { + @Override + protected BalanceAction generate(BalancerClusterState cluster) { + // Move the regions to the servers they were previously hosted on based on the prefetch ratio + if (!oldRegionPrefetchMap.isEmpty() && oldRegionPrefetchMap.entrySet().iterator().hasNext()) { + Map.Entry> regionPrefetchEntry = + oldRegionPrefetchMap.entrySet().iterator().next(); + // Get the server where this region was previously hosted + String regionEncodedName = regionPrefetchEntry.getKey(); + RegionInfo regionInfo = getRegionInfoByEncodedName(cluster, regionEncodedName); + if (regionInfo == null) { + LOG.warn("Region {} not found", regionEncodedName); + oldRegionPrefetchMap.remove(regionEncodedName); + return BalanceAction.NULL_ACTION; + } + if (regionInfo.isMetaRegion() || regionInfo.getTable().isSystemTable()) { + oldRegionPrefetchMap.remove(regionEncodedName); + return BalanceAction.NULL_ACTION; + } + int regionIndex = cluster.regionsToIndex.get(regionInfo); + int oldServerIndex = cluster.serversToIndex + .get(oldRegionPrefetchMap.get(regionEncodedName).getFirst().getAddress()); + if (oldServerIndex < 0) { + LOG.warn("Server previously hosting region {} not found", regionEncodedName); + oldRegionPrefetchMap.remove(regionEncodedName); + return BalanceAction.NULL_ACTION; + } + + float oldPrefetchRatio = + cluster.getOrComputeRegionPrefetchRatio(regionIndex, oldServerIndex); + int currentServerIndex = cluster.regionIndexToServerIndex[regionIndex]; + float currentPrefetchRatio = + cluster.getOrComputeRegionPrefetchRatio(regionIndex, currentServerIndex); + + BalanceAction action = generatePlan(cluster, regionIndex, currentServerIndex, + currentPrefetchRatio, oldServerIndex, oldPrefetchRatio); + oldRegionPrefetchMap.remove(regionEncodedName); + return action; + } + return BalanceAction.NULL_ACTION; + } + + private BalanceAction generatePlan(BalancerClusterState cluster, int regionIndex, + int currentServerIndex, float currentServerPrefetchRatio, int oldServerIndex, + float oldServerPrefetchRatio) { + return moveRegionToOldServer(cluster, regionIndex, currentServerIndex, + currentServerPrefetchRatio, oldServerIndex, oldServerPrefetchRatio) + ? getAction(currentServerIndex, regionIndex, oldServerIndex, -1) + : BalanceAction.NULL_ACTION; + } + + private boolean moveRegionToOldServer(BalancerClusterState cluster, int regionIndex, + int currentServerIndex, float currentServerPrefetchRatio, int oldServerIndex, + float oldServerPrefetchRatio) { + // Find if the region has already moved by comparing the current server index with the + // current server index. This can happen when other candidate generator has moved the region + if (currentServerIndex < 0 || oldServerIndex < 0) { + return false; + } + + float prefetchDiffThreshold = 0.6f; + + // Conditions for moving the region + + // If the region is fully prefetched on the old server, move the region back + if (oldServerPrefetchRatio == 1.0f) { + if (LOG.isDebugEnabled()) { + LOG.debug("Region {} moved to the old server {} as it is fully prefetched there", + cluster.regions[regionIndex].getEncodedName(), cluster.servers[oldServerIndex]); + } + return true; + } + + // Move the region back to the old server if it is prefetched equally on both the servers + if (currentServerPrefetchRatio == oldServerPrefetchRatio) { + if (LOG.isDebugEnabled()) { + LOG.debug("Region {} moved from {} to {} as the prefetch {} is same on both servers", + cluster.regions[regionIndex].getEncodedName(), cluster.servers[currentServerIndex], + cluster.servers[oldServerIndex], currentServerPrefetchRatio); + } + return true; + } + + // If the region is not fully prefetched on either of the servers, move the region back to the + // old server if the current server prefetch is still much less than the old server + if ( + oldServerPrefetchRatio > 0.0f + && currentServerPrefetchRatio / oldServerPrefetchRatio < prefetchDiffThreshold + ) { + if (LOG.isDebugEnabled()) { + LOG.debug( + "Region {} moved from {} to {} old prefetch {} is better than the current " + + "prefetch {}", + cluster.regions[regionIndex].getEncodedName(), cluster.servers[currentServerIndex], + cluster.servers[oldServerIndex], currentServerPrefetchRatio, oldServerPrefetchRatio); + } + return true; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Region {} not moved from {} to {} with current prefetch {} and old prefetch {}", + cluster.regions[regionIndex], cluster.servers[currentServerIndex], + cluster.servers[oldServerIndex], currentServerPrefetchRatio, oldServerPrefetchRatio); + } + return false; + } + } + + private class PrefetchBasedSkewnessCandidateGenerator extends LoadCandidateGenerator { + @Override + BalanceAction pickRandomRegions(BalancerClusterState cluster, int thisServer, int otherServer) { + // First move all the regions which were hosted previously on some other server back to their + // old servers + if (!oldRegionPrefetchMap.isEmpty() && oldRegionPrefetchMap.entrySet().iterator().hasNext()) { + // Get the first region index in the historical prefetch list + Map.Entry> regionEntry = + oldRegionPrefetchMap.entrySet().iterator().next(); + String regionEncodedName = regionEntry.getKey(); + + RegionInfo regionInfo = getRegionInfoByEncodedName(cluster, regionEncodedName); + if (regionInfo == null) { + LOG.warn("Region {} does not exist", regionEncodedName); + oldRegionPrefetchMap.remove(regionEncodedName); + return BalanceAction.NULL_ACTION; + } + if (regionInfo.isMetaRegion() || regionInfo.getTable().isSystemTable()) { + oldRegionPrefetchMap.remove(regionEncodedName); + return BalanceAction.NULL_ACTION; + } + + int regionIndex = cluster.regionsToIndex.get(regionInfo); + + // Get the current host name for this region + thisServer = cluster.regionIndexToServerIndex[regionIndex]; + + // Get the old server index + otherServer = cluster.serversToIndex.get(regionEntry.getValue().getFirst().getAddress()); + + oldRegionPrefetchMap.remove(regionEncodedName); + + if (otherServer < 0) { + // The old server has been moved to other host and hence, the region cannot be moved back + // to the old server + if (LOG.isDebugEnabled()) { + LOG.debug( + "PrefetchBasedSkewnessCandidateGenerator: Region {} not moved to the old " + + "server {} as the server does not exist", + regionEncodedName, regionEntry.getValue().getFirst().getHostname()); + } + return BalanceAction.NULL_ACTION; + } + + if (LOG.isDebugEnabled()) { + LOG.debug( + "PrefetchBasedSkewnessCandidateGenerator: Region {} moved from {} to {} as it " + + "was hosted their earlier", + regionEncodedName, cluster.servers[thisServer].getHostname(), + cluster.servers[otherServer].getHostname()); + } + + return getAction(thisServer, regionIndex, otherServer, -1); + } + + if (thisServer < 0 || otherServer < 0) { + return BalanceAction.NULL_ACTION; + } + + int regionIndexToMove = pickLeastPrefetchedRegion(cluster, thisServer); + if (regionIndexToMove < 0) { + if (LOG.isDebugEnabled()) { + LOG.debug("PrefetchBasedSkewnessCandidateGenerator: No region found for movement"); + } + return BalanceAction.NULL_ACTION; + } + if (LOG.isDebugEnabled()) { + LOG.debug( + "PrefetchBasedSkewnessCandidateGenerator: Region {} moved from {} to {} as it is " + + "least prefetched on current server", + cluster.regions[regionIndexToMove].getEncodedName(), + cluster.servers[thisServer].getHostname(), cluster.servers[otherServer].getHostname()); + } + return getAction(thisServer, regionIndexToMove, otherServer, -1); + } + + private int pickLeastPrefetchedRegion(BalancerClusterState cluster, int thisServer) { + float minPrefetchRatio = Float.MAX_VALUE; + int regionWithMinPrefetch = -1; + for (int i = 0; i < cluster.regionsPerServer[thisServer].length; i++) { + int regionIndex = cluster.regionsPerServer[thisServer][i]; + + float currentPrefetchRatio = + cluster.getOrComputeRegionPrefetchRatio(regionIndex, thisServer); + if (currentPrefetchRatio < minPrefetchRatio) { + minPrefetchRatio = currentPrefetchRatio; + regionWithMinPrefetch = regionIndex; + } + } + return regionWithMinPrefetch; + } + } + + static class PrefetchAwareRegionSkewnessCostFunction extends CostFunction { + static final String REGION_COUNT_SKEW_COST_KEY = + "hbase.master.balancer.stochastic.regionCountCost"; + static final float DEFAULT_REGION_COUNT_SKEW_COST = 20; + private final DoubleArrayCost cost = new DoubleArrayCost(); + + PrefetchAwareRegionSkewnessCostFunction(Configuration conf) { + // Load multiplier should be the greatest as it is the most general way to balance data. + this.setMultiplier(conf.getFloat(REGION_COUNT_SKEW_COST_KEY, DEFAULT_REGION_COUNT_SKEW_COST)); + } + + @Override + void prepare(BalancerClusterState cluster) { + super.prepare(cluster); + cost.prepare(cluster.numServers); + cost.applyCostsChange(costs -> { + for (int i = 0; i < cluster.numServers; i++) { + costs[i] = cluster.regionsPerServer[i].length; + } + }); + } + + @Override + protected double cost() { + return cost.cost(); + } + + @Override + protected void regionMoved(int region, int oldServer, int newServer) { + cost.applyCostsChange(costs -> { + costs[oldServer] = cluster.regionsPerServer[oldServer].length; + costs[newServer] = cluster.regionsPerServer[newServer].length; + }); + } + + public final void updateWeight(double[] weights) { + weights[GeneratorFunctionType.LOAD.ordinal()] += cost(); + } + } + + static class PrefetchCacheCostFunction extends CostFunction { + private static final String PREFETCH_CACHE_COST_KEY = + "hbase.master.balancer.stochastic.prefetchCacheCost"; + private String prefetchedFileListPath; + private double prefetchRatio; + private double bestPrefetchRatio; + + private static final float DEFAULT_PREFETCH_COST = 20; + + PrefetchCacheCostFunction(Configuration conf) { + prefetchedFileListPath = conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY); + // Disable the prefetchCacheCostFunction if the prefetch file list persistence is not enabled + this.setMultiplier(prefetchedFileListPath == null + ? 0.0f + : conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_COST)); + bestPrefetchRatio = 0.0; + prefetchRatio = 0.0; + } + + @Override + void prepare(BalancerClusterState cluster) { + super.prepare(cluster); + prefetchRatio = 0.0; + bestPrefetchRatio = 0.0; + + for (int region = 0; region < cluster.numRegions; region++) { + prefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region, + cluster.regionIndexToServerIndex[region]); + bestPrefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region, + getServerWithBestPrefetchRatioForRegion(region)); + } + + prefetchRatio = bestPrefetchRatio == 0 ? 1.0 : prefetchRatio / bestPrefetchRatio; + if (LOG.isDebugEnabled()) { + LOG.debug("PrefetchCacheCostFunction: Cost: {}", 1 - prefetchRatio); + } + } + + @Override + protected double cost() { + return scale(0, 1, 1 - prefetchRatio); + } + + @Override + protected void regionMoved(int region, int oldServer, int newServer) { + double oldServerPrefetch = cluster.getOrComputeWeightedPrefetchRatio(region, oldServer); + double newServerPrefetch = cluster.getOrComputeWeightedPrefetchRatio(region, newServer); + double prefetchDelta = newServerPrefetch - oldServerPrefetch; + double normalizedDelta = bestPrefetchRatio == 0.0 ? 0.0 : prefetchDelta / bestPrefetchRatio; + prefetchRatio += normalizedDelta; + if (LOG.isDebugEnabled() && (prefetchRatio < 0.0 || prefetchRatio > 1.0)) { + LOG.debug( + "PrefetchCacheCostFunction:regionMoved:region:{}:from:{}:to:{}:oldServerPrefetch:{}:" + + "newServerPrefetch:{}:bestPrefetchRatio:{}:prefetchRatio:{}", + cluster.regions[region].getEncodedName(), cluster.servers[oldServer].getHostname(), + cluster.servers[newServer].getHostname(), oldServerPrefetch, newServerPrefetch, + bestPrefetchRatio, prefetchRatio); + } + } + + private int getServerWithBestPrefetchRatioForRegion(int region) { + return cluster.getOrComputeServerWithBestPrefetchRatio()[region]; + } + + @Override + public final void updateWeight(double[] weights) { + weights[GeneratorFunctionType.PREFETCH.ordinal()] += cost(); + } + } +} 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 edf049e8a718..8316300958e2 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 @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.master.RackManager; import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -136,8 +137,10 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { private long maxRunningTime = DEFAULT_MAX_RUNNING_TIME; private int numRegionLoadsToRemember = DEFAULT_KEEP_REGION_LOADS; private float minCostNeedBalance = DEFAULT_MIN_COST_NEED_BALANCE; + Map> oldRegionPrefetchMap = new HashMap<>(); - private List costFunctions; // FindBugs: Wants this protected; IS2_INCONSISTENT_SYNC + protected List costFunctions; // FindBugs: Wants this protected; + // IS2_INCONSISTENT_SYNC // To save currently configed sum of multiplier. Defaulted at 1 for cases that carry high cost private float sumMultiplier; // to save and report costs to JMX @@ -224,6 +227,24 @@ protected List createCandidateGenerators() { return candidateGenerators; } + protected List createCostFunctions(Configuration conf) { + List costFunctions = new ArrayList<>(); + addCostFunction(costFunctions, new RegionCountSkewCostFunction(conf)); + addCostFunction(costFunctions, new PrimaryRegionCountSkewCostFunction(conf)); + addCostFunction(costFunctions, new MoveCostFunction(conf, provider)); + addCostFunction(costFunctions, localityCost); + addCostFunction(costFunctions, rackLocalityCost); + addCostFunction(costFunctions, new TableSkewCostFunction(conf)); + addCostFunction(costFunctions, regionReplicaHostCostFunction); + addCostFunction(costFunctions, regionReplicaRackCostFunction); + addCostFunction(costFunctions, new ReadRequestCostFunction(conf)); + addCostFunction(costFunctions, new CPRequestCostFunction(conf)); + addCostFunction(costFunctions, new WriteRequestCostFunction(conf)); + addCostFunction(costFunctions, new MemStoreSizeCostFunction(conf)); + addCostFunction(costFunctions, new StoreFileCostFunction(conf)); + return costFunctions; + } + @Override protected void loadConf(Configuration conf) { super.loadConf(conf); @@ -242,20 +263,7 @@ protected void loadConf(Configuration conf) { regionReplicaHostCostFunction = new RegionReplicaHostCostFunction(conf); regionReplicaRackCostFunction = new RegionReplicaRackCostFunction(conf); - costFunctions = new ArrayList<>(); - addCostFunction(new RegionCountSkewCostFunction(conf)); - addCostFunction(new PrimaryRegionCountSkewCostFunction(conf)); - addCostFunction(new MoveCostFunction(conf, provider)); - addCostFunction(localityCost); - addCostFunction(rackLocalityCost); - addCostFunction(new TableSkewCostFunction(conf)); - addCostFunction(regionReplicaHostCostFunction); - addCostFunction(regionReplicaRackCostFunction); - addCostFunction(new ReadRequestCostFunction(conf)); - addCostFunction(new CPRequestCostFunction(conf)); - addCostFunction(new WriteRequestCostFunction(conf)); - addCostFunction(new MemStoreSizeCostFunction(conf)); - addCostFunction(new StoreFileCostFunction(conf)); + this.costFunctions = createCostFunctions(conf); loadCustomCostFunctions(conf); curFunctionCosts = new double[costFunctions.size()]; @@ -460,7 +468,7 @@ protected List balanceTable(TableName tableName, // of all the regions in the table(s) (that's true today) // Keep track of servers to iterate through them. BalancerClusterState cluster = - new BalancerClusterState(loadOfOneTable, loads, finder, rackManager); + new BalancerClusterState(loadOfOneTable, loads, finder, rackManager, oldRegionPrefetchMap); long startTime = EnvironmentEdgeManager.currentTime(); @@ -568,7 +576,7 @@ protected List balanceTable(TableName tableName, return null; } - private void sendRejectionReasonToRingBuffer(Supplier reason, + protected void sendRejectionReasonToRingBuffer(Supplier reason, List costFunctions) { provider.recordBalancerRejection(() -> { BalancerRejection.Builder builder = new BalancerRejection.Builder().setReason(reason.get()); @@ -627,14 +635,14 @@ private void updateStochasticCosts(TableName tableName, double overall, double[] } } - private void addCostFunction(CostFunction costFunction) { + private void addCostFunction(List costFunctions, CostFunction costFunction) { float multiplier = costFunction.getMultiplier(); if (multiplier > 0) { costFunctions.add(costFunction); } } - private String functionCost() { + protected String functionCost() { StringBuilder builder = new StringBuilder(); for (CostFunction c : costFunctions) { builder.append(c.getClass().getSimpleName()); @@ -655,6 +663,12 @@ private String functionCost() { return builder.toString(); } + @RestrictedApi(explanation = "Should only be called in tests", link = "", + allowedOnPath = ".*(/src/test/.*|StochasticLoadBalancer).java") + List getCostFunctions() { + return costFunctions; + } + private String totalCostsPerFunc() { StringBuilder builder = new StringBuilder(); for (CostFunction c : costFunctions) { 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 9ea1c94d1e09..4a996e7796f5 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 @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.NavigableSet; @@ -376,6 +377,19 @@ protected TreeMap> mockClusterServers(int[] mockClu return servers; } + protected Map> mockClusterServersUnsorted(int[] mockCluster, + int numTables) { + int numServers = mockCluster.length; + Map> servers = new LinkedHashMap<>(); + for (int i = 0; i < numServers; i++) { + int numRegions = mockCluster[i]; + ServerAndLoad sal = randomServer(0); + List regions = randomRegions(numRegions, numTables); + servers.put(sal.getServerName(), regions); + } + return servers; + } + protected TreeMap> mockUniformClusterServers(int[] mockCluster) { int numServers = mockCluster.length; TreeMap> servers = new TreeMap<>(); 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 21f3a3b66c9a..cc16cfe2ec83 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 @@ -139,6 +139,8 @@ private ServerMetrics mockServerMetricsWithCpRequests(List regionsOn when(rl.getWriteRequestCount()).thenReturn(0L); when(rl.getMemStoreSize()).thenReturn(Size.ZERO); when(rl.getStoreFileSize()).thenReturn(Size.ZERO); + when(rl.getRegionSizeMB()).thenReturn(Size.ZERO); + when(rl.getCurrentRegionCachedRatio()).thenReturn(0.0f); regionLoadMap.put(info.getRegionName(), rl); } when(serverMetrics.getRegionMetrics()).thenReturn(regionLoadMap); @@ -213,6 +215,8 @@ public void testKeepRegionLoad() throws Exception { when(rl.getWriteRequestCount()).thenReturn(0L); when(rl.getMemStoreSize()).thenReturn(Size.ZERO); when(rl.getStoreFileSize()).thenReturn(new Size(i, Size.Unit.MEGABYTE)); + when(rl.getRegionSizeMB()).thenReturn(Size.ZERO); + when(rl.getCurrentRegionCachedRatio()).thenReturn(0.0f); Map regionLoadMap = new TreeMap<>(Bytes.BYTES_COMPARATOR); regionLoadMap.put(Bytes.toBytes(REGION_KEY), rl); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 12479979b2ba..5ae04048ddbd 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1335,6 +1335,11 @@ public enum OperationStatusCode { */ public static final String BUCKET_CACHE_SIZE_KEY = "hbase.bucketcache.size"; + /** + * Prefetch persistence path key + */ + public static final String PREFETCH_PERSISTENCE_PATH_KEY = "hbase.prefetch.file.list.path"; + /** * HConstants for fast fail on the client side follow */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index 64162bb7464d..92beb8494575 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -1477,6 +1477,7 @@ private void disableCache() { // If persistent ioengine and a path, we will serialize out the backingMap. this.backingMap.clear(); this.fullyCachedFiles.clear(); + this.regionCachedSizeMap.clear(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java index 8f7816106126..e9439affc0cb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java @@ -52,6 +52,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BooleanSupplier; import org.apache.commons.io.FileUtils; @@ -2906,6 +2907,25 @@ public boolean ensureSomeRegionServersAvailable(final int num) throws IOExceptio return startedServer; } + /** + * Waits for all the regions of a table to be prefetched fully + * @param table Table to be wait on. + */ + public void waitForAtleastOneRegionToBePrefetchedOnServer(TableName table, ServerName serverName, + long millisTimeout) throws InterruptedException, IOException { + waitFor(millisTimeout, predicateAtLeastOneRegionIsPrefetchedOnServer(table, serverName)); + } + + public void waitForAllRegionsForTableToBePrefetched(TableName table, long millisTimeout) + throws InterruptedException, IOException { + waitFor(millisTimeout, predicateAllRegionsForTableArePrefetched(table)); + } + + public void waitForMajorityRegionsForTableToBePrefetched(TableName table, long millisTimeout) + throws InterruptedException, IOException { + waitFor(millisTimeout, predicateMajorityRegionsArePrefetched(table)); + } + /** * Make sure that at least the specified number of region servers are running. We don't count the * ones that are currently stopping or are stopped. @@ -3642,6 +3662,110 @@ public boolean evaluate() throws IOException { }; } + /** + * Returns a {@Link Predicate} for checking that all the regions for a table are prefetched + */ + public Waiter.Predicate + predicateAllRegionsForTableArePrefetched(final TableName tableName) { + return new ExplainingPredicate() { + @Override + public String explainFailure() throws IOException { + return "Not all the regions for the table " + tableName.getNameAsString() + + " are prefetched"; + } + + @Override + public boolean evaluate() throws IOException { + List regions = getMiniHBaseCluster().getRegions(tableName); + int totalRegionCount = regions.size(); + AtomicInteger prefetchedRegionCount = new AtomicInteger(); + for (HRegion r : regions) { + getMiniHBaseCluster().getClusterMetrics().getLiveServerMetrics().forEach((sn, sm) -> { + sm.getRegionMetrics().forEach((rn, rm) -> { + String regionNameAsString = r.getRegionInfo().getRegionNameAsString(); + String regionString = rm.getNameAsString(); + if (regionNameAsString.equals(regionString)) { + if (rm.getCurrentRegionCachedRatio() == 1.0f) { + prefetchedRegionCount.getAndIncrement(); + } + } + }); + }); + } + return getAdmin().tableExists(tableName) && totalRegionCount == prefetchedRegionCount.get(); + } + }; + } + + /** + * Returns a {@Link Predicate} for checking that at least one region for the table is prefetched + */ + public Waiter.Predicate predicateAtLeastOneRegionIsPrefetchedOnServer( + final TableName tableName, final ServerName serverName) { + return new ExplainingPredicate() { + @Override + public String explainFailure() throws IOException { + return "No Regions for table " + tableName.getNameAsString() + " prefetched on server " + + serverName.getAddress(); + } + + @Override + public boolean evaluate() throws IOException { + List regions = getMiniHBaseCluster().getRegions(tableName); + AtomicInteger prefetchedRegionCount = new AtomicInteger(); + ServerMetrics sm = + getMiniHBaseCluster().getClusterMetrics().getLiveServerMetrics().get(serverName); + for (HRegion r : regions) { + sm.getRegionMetrics().forEach((rn, rm) -> { + if ( + r.getRegionInfo().getRegionNameAsString().equals(rm.getNameAsString()) + && rm.getCurrentRegionCachedRatio() == 1.0f + ) { + prefetchedRegionCount.getAndIncrement(); + } + }); + } + return getAdmin().tableExists(tableName) && prefetchedRegionCount.get() > 0; + } + }; + } + + /** + * Returns a {@Link Predicate} for checking that more than half of the regions for the table are + * prefetched + */ + public Waiter.Predicate + predicateMajorityRegionsArePrefetched(final TableName tableName) { + return new ExplainingPredicate() { + @Override + public String explainFailure() throws IOException { + return "No Regions for table " + tableName.getNameAsString() + " prefetched"; + } + + @Override + public boolean evaluate() throws IOException { + List regions = getMiniHBaseCluster().getRegions(tableName); + int totalRegionCount = regions.size(); + AtomicInteger prefetchedRegionCount = new AtomicInteger(); + for (HRegion r : regions) { + getMiniHBaseCluster().getClusterMetrics().getLiveServerMetrics().forEach((sn, sm) -> { + sm.getRegionMetrics().forEach((rn, rm) -> { + String regionNameAsString = r.getRegionInfo().getRegionNameAsString(); + String regionString = rm.getNameAsString(); + if (regionNameAsString.equals(regionString)) { + if (rm.getCurrentRegionCachedRatio() == 1.0f) { + prefetchedRegionCount.getAndIncrement(); + } + } + }); + }); + } + return getAdmin().tableExists(tableName) + && (float) prefetchedRegionCount.get() / totalRegionCount > 0.5f; + } + }; + } + /** * Wait until no regions in transition. * @param timeout How long to wait. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockEvictionOnRegionMovement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockEvictionOnRegionMovement.java index eb3e3cc61f4b..b6818ad4b62f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockEvictionOnRegionMovement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockEvictionOnRegionMovement.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.io.hfile; import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_IOENGINE_KEY; +import static org.apache.hadoop.hbase.HConstants.PREFETCH_PERSISTENCE_PATH_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchRSClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchRSClose.java index 64db9158333d..86665d3da66f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchRSClose.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchRSClose.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.io.hfile; import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_IOENGINE_KEY; +import static org.apache.hadoop.hbase.HConstants.PREFETCH_PERSISTENCE_PATH_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancer.java new file mode 100644 index 000000000000..fd491fa88e68 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancer.java @@ -0,0 +1,403 @@ +/* + * 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 static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ClusterMetrics; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.RegionMetrics; +import org.apache.hadoop.hbase.ServerMetrics; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.Size; +import org.apache.hadoop.hbase.TableName; +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.master.RegionPlan; +import org.apache.hadoop.hbase.testclassification.LargeTests; +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; + +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + +@Category({ LargeTests.class }) +public class TestPrefetchAwareLoadBalancer extends BalancerTestBase { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPrefetchAwareLoadBalancer.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestPrefetchAwareLoadBalancer.class); + + private static PrefetchAwareLoadBalancer loadBalancer; + + static List servers; + + static List tableDescs; + + static Map tableMap = new HashMap<>(); + + static TableName[] tables = new TableName[] { TableName.valueOf("dt1"), TableName.valueOf("dt2"), + TableName.valueOf("dt3"), TableName.valueOf("dt4") }; + + private static List generateServers(int numServers) { + List servers = new ArrayList<>(numServers); + Random rand = ThreadLocalRandom.current(); + for (int i = 0; i < numServers; i++) { + String host = "server" + rand.nextInt(100000); + int port = rand.nextInt(60000); + servers.add(ServerName.valueOf(host, port, -1)); + } + return servers; + } + + private static List constructTableDesc(boolean hasBogusTable) { + List tds = Lists.newArrayList(); + for (int i = 0; i < tables.length; i++) { + TableDescriptor htd = TableDescriptorBuilder.newBuilder(tables[i]).build(); + tds.add(htd); + } + return tds; + } + + private ServerMetrics mockServerMetricsWithRegionPrefetchInfo(ServerName server, + List regionsOnServer, float currentPrefetchRatio, + List oldPrefechedRegions, int oldPrefetchSize, int regionSize) { + ServerMetrics serverMetrics = mock(ServerMetrics.class); + Map regionLoadMap = new TreeMap<>(Bytes.BYTES_COMPARATOR); + for (RegionInfo info : regionsOnServer) { + RegionMetrics rl = mock(RegionMetrics.class); + when(rl.getReadRequestCount()).thenReturn(0L); + when(rl.getWriteRequestCount()).thenReturn(0L); + when(rl.getMemStoreSize()).thenReturn(Size.ZERO); + when(rl.getStoreFileSize()).thenReturn(Size.ZERO); + when(rl.getCurrentRegionCachedRatio()).thenReturn(currentPrefetchRatio); + when(rl.getRegionSizeMB()).thenReturn(new Size(regionSize, Size.Unit.MEGABYTE)); + regionLoadMap.put(info.getRegionName(), rl); + } + when(serverMetrics.getRegionMetrics()).thenReturn(regionLoadMap); + Map oldPrefetchInfoMap = new HashMap<>(); + for (RegionInfo info : oldPrefechedRegions) { + oldPrefetchInfoMap.put(info.getEncodedName(), oldPrefetchSize); + } + when(serverMetrics.getRegionCachedInfo()).thenReturn(oldPrefetchInfoMap); + return serverMetrics; + } + + @BeforeClass + public static void beforeAllTests() throws Exception { + servers = generateServers(3); + tableDescs = constructTableDesc(false); + Configuration conf = HBaseConfiguration.create(); + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "prefetch_file_list"); + loadBalancer = new PrefetchAwareLoadBalancer(); + loadBalancer.setClusterInfoProvider(new DummyClusterInfoProvider(conf)); + loadBalancer.loadConf(conf); + } + + @Test + public void testRegionsNotPrefetchedOnOldServerAndCurrentServer() throws Exception { + // The regions are not prefetched on old server as well as the current server. This causes + // skewness in the region allocation which should be fixed by the balancer + + Map> clusterState = new HashMap<>(); + ServerName server0 = servers.get(0); + ServerName server1 = servers.get(1); + ServerName server2 = servers.get(2); + + // Simulate that the regions previously hosted by server1 are now hosted on server0 + List regionsOnServer0 = randomRegions(10); + List regionsOnServer1 = randomRegions(0); + List regionsOnServer2 = randomRegions(5); + + clusterState.put(server0, regionsOnServer0); + clusterState.put(server1, regionsOnServer1); + clusterState.put(server2, regionsOnServer2); + + // Mock cluster metrics + Map serverMetricsMap = new TreeMap<>(); + serverMetricsMap.put(server0, mockServerMetricsWithRegionPrefetchInfo(server0, regionsOnServer0, + 0.0f, new ArrayList<>(), 0, 10)); + serverMetricsMap.put(server1, mockServerMetricsWithRegionPrefetchInfo(server1, regionsOnServer1, + 0.0f, new ArrayList<>(), 0, 10)); + serverMetricsMap.put(server2, mockServerMetricsWithRegionPrefetchInfo(server2, regionsOnServer2, + 0.0f, new ArrayList<>(), 0, 10)); + ClusterMetrics clusterMetrics = mock(ClusterMetrics.class); + when(clusterMetrics.getLiveServerMetrics()).thenReturn(serverMetricsMap); + loadBalancer.updateClusterMetrics(clusterMetrics); + + Map>> LoadOfAllTable = + (Map) mockClusterServersWithTables(clusterState); + List plans = loadBalancer.balanceCluster(LoadOfAllTable); + Set regionsMovedFromServer0 = new HashSet<>(); + Map> targetServers = new HashMap<>(); + for (RegionPlan plan : plans) { + if (plan.getSource().equals(server0)) { + regionsMovedFromServer0.add(plan.getRegionInfo()); + if (!targetServers.containsKey(plan.getDestination())) { + targetServers.put(plan.getDestination(), new ArrayList<>()); + } + targetServers.get(plan.getDestination()).add(plan.getRegionInfo()); + } + } + // should move 5 regions from server0 to server 1 + assertEquals(5, regionsMovedFromServer0.size()); + assertEquals(5, targetServers.get(server1).size()); + } + + @Test + public void testRegionsPartiallyPrefetchedOnOldServerAndNotPrefetchedOnCurrentServer() + throws Exception { + // The regions are partially prefetched on old server but not prefetched on the current server + + Map> clusterState = new HashMap<>(); + ServerName server0 = servers.get(0); + ServerName server1 = servers.get(1); + ServerName server2 = servers.get(2); + + // Simulate that the regions previously hosted by server1 are now hosted on server0 + List regionsOnServer0 = randomRegions(10); + List regionsOnServer1 = randomRegions(0); + List regionsOnServer2 = randomRegions(5); + + clusterState.put(server0, regionsOnServer0); + clusterState.put(server1, regionsOnServer1); + clusterState.put(server2, regionsOnServer2); + + // Mock cluster metrics + + // Mock 5 regions from server0 were previously hosted on server1 + List oldPrefetchedRegions = + regionsOnServer0.subList(5, regionsOnServer0.size() - 1); + + Map serverMetricsMap = new TreeMap<>(); + serverMetricsMap.put(server0, mockServerMetricsWithRegionPrefetchInfo(server0, regionsOnServer0, + 0.0f, new ArrayList<>(), 0, 10)); + serverMetricsMap.put(server1, mockServerMetricsWithRegionPrefetchInfo(server1, regionsOnServer1, + 0.0f, oldPrefetchedRegions, 6, 10)); + serverMetricsMap.put(server2, mockServerMetricsWithRegionPrefetchInfo(server2, regionsOnServer2, + 0.0f, new ArrayList<>(), 0, 10)); + ClusterMetrics clusterMetrics = mock(ClusterMetrics.class); + when(clusterMetrics.getLiveServerMetrics()).thenReturn(serverMetricsMap); + loadBalancer.updateClusterMetrics(clusterMetrics); + + Map>> LoadOfAllTable = + (Map) mockClusterServersWithTables(clusterState); + List plans = loadBalancer.balanceCluster(LoadOfAllTable); + Set regionsMovedFromServer0 = new HashSet<>(); + Map> targetServers = new HashMap<>(); + for (RegionPlan plan : plans) { + if (plan.getSource().equals(server0)) { + regionsMovedFromServer0.add(plan.getRegionInfo()); + if (!targetServers.containsKey(plan.getDestination())) { + targetServers.put(plan.getDestination(), new ArrayList<>()); + } + targetServers.get(plan.getDestination()).add(plan.getRegionInfo()); + } + } + // should move 5 regions from server0 to server1 + assertEquals(5, regionsMovedFromServer0.size()); + assertEquals(5, targetServers.get(server1).size()); + assertTrue(targetServers.get(server1).containsAll(oldPrefetchedRegions)); + } + + @Test + public void testRegionsFullyPrefetchedOnOldServerAndNoPrefetchedOnCurrentServers() + throws Exception { + // The regions are fully prefetched on old server + + Map> clusterState = new HashMap<>(); + ServerName server0 = servers.get(0); + ServerName server1 = servers.get(1); + ServerName server2 = servers.get(2); + + // Simulate that the regions previously hosted by server1 are now hosted on server0 + List regionsOnServer0 = randomRegions(10); + List regionsOnServer1 = randomRegions(0); + List regionsOnServer2 = randomRegions(5); + + clusterState.put(server0, regionsOnServer0); + clusterState.put(server1, regionsOnServer1); + clusterState.put(server2, regionsOnServer2); + + // Mock cluster metrics + + // Mock 5 regions from server0 were previously hosted on server1 + List oldPrefetchedRegions = + regionsOnServer0.subList(5, regionsOnServer0.size() - 1); + + Map serverMetricsMap = new TreeMap<>(); + serverMetricsMap.put(server0, mockServerMetricsWithRegionPrefetchInfo(server0, regionsOnServer0, + 0.0f, new ArrayList<>(), 0, 10)); + serverMetricsMap.put(server1, mockServerMetricsWithRegionPrefetchInfo(server1, regionsOnServer1, + 0.0f, oldPrefetchedRegions, 10, 10)); + serverMetricsMap.put(server2, mockServerMetricsWithRegionPrefetchInfo(server2, regionsOnServer2, + 0.0f, new ArrayList<>(), 0, 10)); + ClusterMetrics clusterMetrics = mock(ClusterMetrics.class); + when(clusterMetrics.getLiveServerMetrics()).thenReturn(serverMetricsMap); + loadBalancer.updateClusterMetrics(clusterMetrics); + + Map>> LoadOfAllTable = + (Map) mockClusterServersWithTables(clusterState); + List plans = loadBalancer.balanceCluster(LoadOfAllTable); + Set regionsMovedFromServer0 = new HashSet<>(); + Map> targetServers = new HashMap<>(); + for (RegionPlan plan : plans) { + if (plan.getSource().equals(server0)) { + regionsMovedFromServer0.add(plan.getRegionInfo()); + if (!targetServers.containsKey(plan.getDestination())) { + targetServers.put(plan.getDestination(), new ArrayList<>()); + } + targetServers.get(plan.getDestination()).add(plan.getRegionInfo()); + } + } + // should move 5 regions from server0 to server1 + assertEquals(5, regionsMovedFromServer0.size()); + assertEquals(5, targetServers.get(server1).size()); + assertTrue(targetServers.get(server1).containsAll(oldPrefetchedRegions)); + } + + @Test + public void testRegionsFullyPrefetchedOnOldAndCurrentServers() throws Exception { + // The regions are fully prefetched on old server + + Map> clusterState = new HashMap<>(); + ServerName server0 = servers.get(0); + ServerName server1 = servers.get(1); + ServerName server2 = servers.get(2); + + // Simulate that the regions previously hosted by server1 are now hosted on server0 + List regionsOnServer0 = randomRegions(10); + List regionsOnServer1 = randomRegions(0); + List regionsOnServer2 = randomRegions(5); + + clusterState.put(server0, regionsOnServer0); + clusterState.put(server1, regionsOnServer1); + clusterState.put(server2, regionsOnServer2); + + // Mock cluster metrics + + // Mock 5 regions from server0 were previously hosted on server1 + List oldPrefetchedRegions = + regionsOnServer0.subList(5, regionsOnServer0.size() - 1); + + Map serverMetricsMap = new TreeMap<>(); + serverMetricsMap.put(server0, mockServerMetricsWithRegionPrefetchInfo(server0, regionsOnServer0, + 1.0f, new ArrayList<>(), 0, 10)); + serverMetricsMap.put(server1, mockServerMetricsWithRegionPrefetchInfo(server1, regionsOnServer1, + 1.0f, oldPrefetchedRegions, 10, 10)); + serverMetricsMap.put(server2, mockServerMetricsWithRegionPrefetchInfo(server2, regionsOnServer2, + 1.0f, new ArrayList<>(), 0, 10)); + ClusterMetrics clusterMetrics = mock(ClusterMetrics.class); + when(clusterMetrics.getLiveServerMetrics()).thenReturn(serverMetricsMap); + loadBalancer.updateClusterMetrics(clusterMetrics); + + Map>> LoadOfAllTable = + (Map) mockClusterServersWithTables(clusterState); + List plans = loadBalancer.balanceCluster(LoadOfAllTable); + Set regionsMovedFromServer0 = new HashSet<>(); + Map> targetServers = new HashMap<>(); + for (RegionPlan plan : plans) { + if (plan.getSource().equals(server0)) { + regionsMovedFromServer0.add(plan.getRegionInfo()); + if (!targetServers.containsKey(plan.getDestination())) { + targetServers.put(plan.getDestination(), new ArrayList<>()); + } + targetServers.get(plan.getDestination()).add(plan.getRegionInfo()); + } + } + // should move 5 regions from server0 to server1 + assertEquals(5, regionsMovedFromServer0.size()); + assertEquals(5, targetServers.get(server1).size()); + assertTrue(targetServers.get(server1).containsAll(oldPrefetchedRegions)); + } + + @Test + public void testRegionsPartiallyPrefetchedOnOldServerAndCurrentServer() throws Exception { + // The regions are partially prefetched on old server + + Map> clusterState = new HashMap<>(); + ServerName server0 = servers.get(0); + ServerName server1 = servers.get(1); + ServerName server2 = servers.get(2); + + // Simulate that the regions previously hosted by server1 are now hosted on server0 + List regionsOnServer0 = randomRegions(10); + List regionsOnServer1 = randomRegions(0); + List regionsOnServer2 = randomRegions(5); + + clusterState.put(server0, regionsOnServer0); + clusterState.put(server1, regionsOnServer1); + clusterState.put(server2, regionsOnServer2); + + // Mock cluster metrics + + // Mock 5 regions from server0 were previously hosted on server1 + List oldPrefetchedRegions = + regionsOnServer0.subList(5, regionsOnServer0.size() - 1); + + Map serverMetricsMap = new TreeMap<>(); + serverMetricsMap.put(server0, mockServerMetricsWithRegionPrefetchInfo(server0, regionsOnServer0, + 0.2f, new ArrayList<>(), 0, 10)); + serverMetricsMap.put(server1, mockServerMetricsWithRegionPrefetchInfo(server1, regionsOnServer1, + 0.0f, oldPrefetchedRegions, 6, 10)); + serverMetricsMap.put(server2, mockServerMetricsWithRegionPrefetchInfo(server2, regionsOnServer2, + 1.0f, new ArrayList<>(), 0, 10)); + ClusterMetrics clusterMetrics = mock(ClusterMetrics.class); + when(clusterMetrics.getLiveServerMetrics()).thenReturn(serverMetricsMap); + loadBalancer.updateClusterMetrics(clusterMetrics); + + Map>> LoadOfAllTable = + (Map) mockClusterServersWithTables(clusterState); + List plans = loadBalancer.balanceCluster(LoadOfAllTable); + Set regionsMovedFromServer0 = new HashSet<>(); + Map> targetServers = new HashMap<>(); + for (RegionPlan plan : plans) { + if (plan.getSource().equals(server0)) { + regionsMovedFromServer0.add(plan.getRegionInfo()); + if (!targetServers.containsKey(plan.getDestination())) { + targetServers.put(plan.getDestination(), new ArrayList<>()); + } + targetServers.get(plan.getDestination()).add(plan.getRegionInfo()); + } + } + assertEquals(5, regionsMovedFromServer0.size()); + assertEquals(5, targetServers.get(server1).size()); + assertTrue(targetServers.get(server1).containsAll(oldPrefetchedRegions)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancerCostFunctions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancerCostFunctions.java new file mode 100644 index 000000000000..9568d78c1577 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancerCostFunctions.java @@ -0,0 +1,319 @@ +/* + * 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.assertTrue; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Pair; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MasterTests.class, MediumTests.class }) +public class TestPrefetchAwareLoadBalancerCostFunctions extends StochasticBalancerTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPrefetchAwareLoadBalancerCostFunctions.class); + + // Mapping of prefetch test -> expected prefetch + private final float[] expectedPrefetch = { 0.0f, 0.0f, 0.5f, 1.0f, 0.0f, 0.572f, 0.0f, 0.075f }; + + /** + * Data set to testPrefetchCost: [test][0][0] = mapping of server to number of regions it hosts + * [test][region + 1][0] = server that region is hosted on [test][region + 1][server + 1] = + * prefetch of that region on server + */ + private final int[][][] clusterRegionPrefetchMocks = new int[][][] { + // Test 1: each region is entirely on server that hosts it + // Cost of moving the regions in this case should be high as the regions are fully prefetched + // on the server they are currently hosted on + new int[][] { new int[] { 2, 1, 1 }, // Server 0 has 2, server 1 has 1 and server 2 has 1 + // region(s) hosted respectively + new int[] { 0, 100, 0, 0 }, // region 0 is hosted and prefetched only on server 0 + new int[] { 0, 100, 0, 0 }, // region 1 is hosted and prefetched only on server 0 + new int[] { 1, 0, 100, 0 }, // region 2 is hosted and prefetched only on server 1 + new int[] { 2, 0, 0, 100 }, // region 3 is hosted and prefetched only on server 2 + }, + + // Test 2: each region is prefetched completely on the server it is currently hosted on, + // but it was also prefetched on some other server historically + // Cost of moving the regions in this case should be high as the regions are fully prefetched + // on the server they are currently hosted on. Although, the regions were previously hosted and + // prefetched on some other server, since they are completely prefetched on the new server, + // there is no need to move the regions back to the previously hosting cluster + new int[][] { new int[] { 1, 2, 1 }, // Server 0 has 1, server 1 has 2 and server 2 has 1 + // region(s) hosted respectively + new int[] { 0, 100, 0, 100 }, // region 0 is hosted and currently prefetched on server 0, + // but previously prefetched completely on server 2 + new int[] { 1, 100, 100, 0 }, // region 1 is hosted and currently prefetched on server 1, + // but previously prefetched completely on server 0 + new int[] { 1, 0, 100, 100 }, // region 2 is hosted and currently prefetched on server 1, + // but previously prefetched on server 2 + new int[] { 2, 0, 100, 100 }, // region 3 is hosted and currently prefetched on server 2, + // but previously prefetched on server 1 + }, + + // Test 3: The regions were hosted and fully prefetched on a server but later moved to other + // because of server crash procedure. The regions are partially prefetched on the server they + // are currently hosted on + new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 50, 0, 100 }, // Region 0 is currently + // hosted and partially + // prefetched on + // server 0, but was fully + // prefetched on server 2 + // previously + new int[] { 1, 100, 50, 0 }, // Region 1 is currently hosted and partially prefetched on + // server 1, but was fully prefetched on server 0 previously + new int[] { 1, 0, 50, 100 }, // Region 2 is currently hosted and partially prefetched on + // server 1, but was fully prefetched on server 2 previously + new int[] { 2, 0, 100, 50 }, // Region 3 is currently hosted and partially prefetched on + // server 2, but was fully prefetched on server 1 previously + }, + + // Test 4: The regions were hosted and fully prefetched on a server, but later moved to other + // server because of server crash procedure. The regions are not at all prefetched on the server + // they are currently hosted on + new int[][] { new int[] { 1, 1, 2 }, new int[] { 0, 0, 0, 100 }, // Region 0 is currently hosted + // but not prefetched on server + // 0, + // but was fully prefetched on + // server 2 previously + new int[] { 1, 100, 0, 0 }, // Region 1 is currently hosted but not prefetched on server 1, + // but was fully prefetched on server 0 previously + new int[] { 2, 0, 100, 0 }, // Region 2 is currently hosted but not prefetched on server 2, + // but was fully prefetched on server 1 previously + new int[] { 2, 100, 0, 0 }, // Region 3 is currently hosted but not prefetched on server 2, + // but was fully prefetched on server 1 previously + }, + + // Test 5: The regions were partially prefetched on old servers, before moving to the new server + // where also, they are partially prefetched + new int[][] { new int[] { 2, 1, 1 }, new int[] { 0, 50, 50, 0 }, // Region 0 is hosted and + // partially prefetched on + // server 0, but + // was previously hosted and + // partially prefetched on + // server 1 + new int[] { 0, 50, 0, 50 }, // Region 1 is hosted and partially prefetched on server 0, but + // was previously hosted and partially prefetched on server 2 + new int[] { 1, 0, 50, 50 }, // Region 2 is hosted and partially prefetched on server 1, but + // was previously hosted and partially prefetched on server 2 + new int[] { 2, 0, 50, 50 }, // Region 3 is hosted and partially prefetched on server 2, but + // was previously hosted and partially prefetched on server 1 + }, + + // Test 6: The regions are less prefetched on the new servers as compared to what they were + // prefetched on the server before they were moved to the new servers + new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 30, 70, 0 }, // Region 0 is hosted and + // prefetched 30% on server 0, + // but was + // previously hosted and + // prefetched 70% on server 1 + new int[] { 1, 70, 30, 0 }, // Region 1 is hosted and prefetched 30% on server 1, but was + // previously hosted and prefetched 70% on server 0 + new int[] { 1, 0, 30, 70 }, // Region 2 is hosted and prefetched 30% on server 1, but was + // previously hosted and prefetched 70% on server 2 + new int[] { 2, 0, 70, 30 }, // Region 3 is hosted and prefetched 30% on server 2, but was + // previously hosted and prefetched 70% on server 1 + }, + + // Test 7: The regions are more prefetched on the new servers as compared to what they were + // prefetched on the server before they were moved to the new servers + new int[][] { new int[] { 2, 1, 1 }, new int[] { 0, 80, 20, 0 }, // Region 0 is hosted and 80% + // prefetched on server 0, but + // was + // previously hosted and 20% + // prefetched on server 1 + new int[] { 0, 80, 0, 20 }, // Region 1 is hosted and 80% prefetched on server 0, but was + // previously hosted and 20% prefetched on server 2 + new int[] { 1, 20, 80, 0 }, // Region 2 is hosted and 80% prefetched on server 1, but was + // previously hosted and 20% prefetched on server 0 + new int[] { 2, 0, 20, 80 }, // Region 3 is hosted and 80% prefetched on server 2, but was + // previously hosted and 20% prefetched on server 1 + }, + + // Test 8: The regions are randomly assigned to the server with some regions historically + // hosted on other region servers + new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 34, 0, 58 }, // Region 0 is hosted and + // partially prefetched on + // server 0, + // but was previously hosted + // and partially prefetched on + // server 2 + // current prefetch < + // historical prefetch + new int[] { 1, 78, 100, 0 }, // Region 1 is hosted and fully prefetched on server 1, + // but was previously hosted and partially prefetched on server 0 + // current prefetch > historical prefetch + new int[] { 1, 66, 66, 0 }, // Region 2 is hosted and partially prefetched on server 1, + // but was previously hosted and partially prefetched on server 0 + // current prefetch == historical prefetch + new int[] { 2, 0, 0, 96 }, // Region 3 is hosted and partially prefetched on server 0 + // No historical prefetch + }, }; + + private static Configuration storedConfiguration; + + private PrefetchAwareLoadBalancer loadBalancer = new PrefetchAwareLoadBalancer(); + + @BeforeClass + public static void saveInitialConfiguration() { + storedConfiguration = new Configuration(conf); + } + + @Before + public void beforeEachTest() { + conf = new Configuration(storedConfiguration); + loadBalancer.loadConf(conf); + } + + @Test + public void testVerifyPrefetchAwareSkewnessCostFunctionEnabled() { + PrefetchAwareLoadBalancer lb = new PrefetchAwareLoadBalancer(); + lb.loadConf(conf); + assertTrue(Arrays.asList(lb.getCostFunctionNames()).contains( + PrefetchAwareLoadBalancer.PrefetchAwareRegionSkewnessCostFunction.class.getSimpleName())); + } + + @Test + public void testVerifyPrefetchAwareSkewnessCostFunctionDisabled() { + conf.setFloat( + PrefetchAwareLoadBalancer.PrefetchAwareRegionSkewnessCostFunction.REGION_COUNT_SKEW_COST_KEY, + 0.0f); + + PrefetchAwareLoadBalancer lb = new PrefetchAwareLoadBalancer(); + lb.loadConf(conf); + + assertFalse(Arrays.asList(lb.getCostFunctionNames()).contains( + PrefetchAwareLoadBalancer.PrefetchAwareRegionSkewnessCostFunction.class.getSimpleName())); + } + + @Test + public void testVerifyPrefetchCostFunctionEnabled() { + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch.persistence"); + + PrefetchAwareLoadBalancer lb = new PrefetchAwareLoadBalancer(); + lb.loadConf(conf); + + assertTrue(Arrays.asList(lb.getCostFunctionNames()) + .contains(PrefetchAwareLoadBalancer.PrefetchCacheCostFunction.class.getSimpleName())); + } + + @Test + public void testVerifyPrefetchCostFunctionDisabledByNoPersistencePathKey() { + assertFalse(Arrays.asList(loadBalancer.getCostFunctionNames()) + .contains(PrefetchAwareLoadBalancer.PrefetchCacheCostFunction.class.getSimpleName())); + } + + @Test + public void testVerifyPrefetchCostFunctionDisabledByNoMultiplier() { + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch.persistence"); + conf.setFloat("hbase.master.balancer.stochastic.prefetchCacheCost", 0.0f); + assertFalse(Arrays.asList(loadBalancer.getCostFunctionNames()) + .contains(PrefetchAwareLoadBalancer.PrefetchCacheCostFunction.class.getSimpleName())); + } + + @Test + public void testPrefetchCost() { + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch.persistence"); + PrefetchAwareLoadBalancer.PrefetchCacheCostFunction costFunction = + new PrefetchAwareLoadBalancer.PrefetchCacheCostFunction(conf); + + for (int test = 0; test < clusterRegionPrefetchMocks.length; test++) { + int[][] clusterRegionLocations = clusterRegionPrefetchMocks[test]; + TestPrefetchAwareLoadBalancerCostFunctions.MockClusterForPrefetch cluster = + new TestPrefetchAwareLoadBalancerCostFunctions.MockClusterForPrefetch( + clusterRegionLocations); + costFunction.prepare(cluster); + double cost = costFunction.cost(); + assertEquals(expectedPrefetch[test], cost, 0.01); + } + } + + private class MockClusterForPrefetch extends BalancerClusterState { + private final Map, Float> regionServerPrefetch = new HashMap<>(); + + public MockClusterForPrefetch(int[][] regionsArray) { + // regions[0] is an array where index = serverIndex and value = number of regions + super(mockClusterServersUnsorted(regionsArray[0], 1), null, null, null, null); + Map> oldPrefetchRatio = new HashMap<>(); + for (int i = 1; i < regionsArray.length; i++) { + int regionIndex = i - 1; + for (int j = 1; j < regionsArray[i].length; j++) { + int serverIndex = j - 1; + float prefetch = (float) regionsArray[i][j] / 100; + regionServerPrefetch.put(new Pair<>(regionIndex, serverIndex), prefetch); + if (prefetch > 0.0f && serverIndex != regionsArray[i][0]) { + // This is the historical prefetch value + oldPrefetchRatio.put(regions[regionIndex].getEncodedName(), + new Pair<>(servers[serverIndex], prefetch)); + } + } + } + oldRegionServerPrefetchRatio = oldPrefetchRatio; + } + + @Override + public int getTotalRegionHFileSizeMB(int region) { + return 1; + } + + @Override + protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) { + float prefetchRatio = 0.0f; + + // Get the prefetch cache ratio if the region is currently hosted on this server + if (regionServerIndex == regionIndexToServerIndex[region]) { + return regionServerPrefetch.get(new Pair<>(region, regionServerIndex)); + } + + // Region is not currently hosted on this server. Check if the region was prefetched on this + // server earlier. This can happen when the server was shutdown and the cache was persisted. + // Search using the index name and server name and not the index id and server id as these + // ids may change when a server is marked as dead or a new server is added. + String regionEncodedName = regions[region].getEncodedName(); + ServerName serverName = servers[regionServerIndex]; + if ( + oldRegionServerPrefetchRatio != null + && oldRegionServerPrefetchRatio.containsKey(regionEncodedName) + ) { + Pair serverPrefetchRatio = + oldRegionServerPrefetchRatio.get(regionEncodedName); + if (ServerName.isSameAddress(serverName, serverPrefetchRatio.getFirst())) { + prefetchRatio = serverPrefetchRatio.getSecond(); + oldRegionServerPrefetchRatio.remove(regionEncodedName); + } + } + return prefetchRatio; + } + } +} From 755427cb647ce1f63dc3152f90befc7fad98a10d Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Tue, 29 Aug 2023 21:27:05 +0530 Subject: [PATCH 2/5] HBASE-27999 Implement cache prefetch aware load balancer Fixed spotless and spotbugs errors. --- .../hbase/master/balancer/PrefetchAwareLoadBalancer.java | 4 ++-- .../hbase/io/hfile/TestBlockEvictionOnRegionMovement.java | 1 - .../org/apache/hadoop/hbase/io/hfile/TestPrefetchRSClose.java | 1 - 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java index 8de37f3158c8..632f006259ba 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java @@ -90,7 +90,7 @@ private void addCostFunction(List costFunctions, CostFunction cost } @Override - public synchronized void updateClusterMetrics(ClusterMetrics clusterMetrics) { + public void updateClusterMetrics(ClusterMetrics clusterMetrics) { this.clusterStatus = clusterMetrics; updateRegionLoad(); } @@ -98,7 +98,7 @@ public synchronized void updateClusterMetrics(ClusterMetrics clusterMetrics) { /** * Collect the prefetch information for all the regions from all the active region servers. */ - private synchronized void updateRegionLoad() { + private void updateRegionLoad() { loads = new HashMap<>(); oldRegionPrefetchMap = new HashMap<>(); Map> currentPrefetchMap = new HashMap<>(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockEvictionOnRegionMovement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockEvictionOnRegionMovement.java index b6818ad4b62f..eb3e3cc61f4b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockEvictionOnRegionMovement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockEvictionOnRegionMovement.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.io.hfile; import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_IOENGINE_KEY; -import static org.apache.hadoop.hbase.HConstants.PREFETCH_PERSISTENCE_PATH_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchRSClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchRSClose.java index 86665d3da66f..64db9158333d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchRSClose.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchRSClose.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.io.hfile; import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_IOENGINE_KEY; -import static org.apache.hadoop.hbase.HConstants.PREFETCH_PERSISTENCE_PATH_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; From 0eefe7915062774e3402c828c44cdaf9f6828341 Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Wed, 30 Aug 2023 10:37:24 +0530 Subject: [PATCH 3/5] HBASE-27999 Implement cache prefetch aware load balancer Fixed test failures --- ...upBasedLoadBalancerWithStochasticLoadBalancerAsInternal.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancerWithStochasticLoadBalancerAsInternal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancerWithStochasticLoadBalancerAsInternal.java index 748045246b3b..67ef296da58b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancerWithStochasticLoadBalancerAsInternal.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancerWithStochasticLoadBalancerAsInternal.java @@ -86,6 +86,8 @@ private ServerMetrics mockServerMetricsWithReadRequests(ServerName server, when(rl.getWriteRequestCount()).thenReturn(0L); when(rl.getMemStoreSize()).thenReturn(Size.ZERO); when(rl.getStoreFileSize()).thenReturn(Size.ZERO); + when(rl.getRegionSizeMB()).thenReturn(Size.ZERO); + when(rl.getCurrentRegionCachedRatio()).thenReturn(0.0f); regionLoadMap.put(info.getRegionName(), rl); } when(serverMetrics.getRegionMetrics()).thenReturn(regionLoadMap); From ebecb95dbb419695ae1efb45ea6a309532890870 Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Wed, 30 Aug 2023 16:52:35 +0530 Subject: [PATCH 4/5] HBASE-27999 Implement cache prefetch aware load balancer --- .../hbase/io/hfile/bucket/BucketCache.java | 1 + .../apache/hadoop/hbase/HBaseTestingUtil.java | 124 ------------------ 2 files changed, 1 insertion(+), 124 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index 92beb8494575..0faf510f5db6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -342,6 +342,7 @@ public BucketCache(String ioEngineName, long capacity, int blockSize, int[] buck } catch (IOException ioex) { backingMap.clear(); fullyCachedFiles.clear(); + regionCachedSizeMap.clear(); LOG.error("Can't restore from file[" + persistencePath + "] because of ", ioex); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java index e9439affc0cb..8f7816106126 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java @@ -52,7 +52,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BooleanSupplier; import org.apache.commons.io.FileUtils; @@ -2907,25 +2906,6 @@ public boolean ensureSomeRegionServersAvailable(final int num) throws IOExceptio return startedServer; } - /** - * Waits for all the regions of a table to be prefetched fully - * @param table Table to be wait on. - */ - public void waitForAtleastOneRegionToBePrefetchedOnServer(TableName table, ServerName serverName, - long millisTimeout) throws InterruptedException, IOException { - waitFor(millisTimeout, predicateAtLeastOneRegionIsPrefetchedOnServer(table, serverName)); - } - - public void waitForAllRegionsForTableToBePrefetched(TableName table, long millisTimeout) - throws InterruptedException, IOException { - waitFor(millisTimeout, predicateAllRegionsForTableArePrefetched(table)); - } - - public void waitForMajorityRegionsForTableToBePrefetched(TableName table, long millisTimeout) - throws InterruptedException, IOException { - waitFor(millisTimeout, predicateMajorityRegionsArePrefetched(table)); - } - /** * Make sure that at least the specified number of region servers are running. We don't count the * ones that are currently stopping or are stopped. @@ -3662,110 +3642,6 @@ public boolean evaluate() throws IOException { }; } - /** - * Returns a {@Link Predicate} for checking that all the regions for a table are prefetched - */ - public Waiter.Predicate - predicateAllRegionsForTableArePrefetched(final TableName tableName) { - return new ExplainingPredicate() { - @Override - public String explainFailure() throws IOException { - return "Not all the regions for the table " + tableName.getNameAsString() - + " are prefetched"; - } - - @Override - public boolean evaluate() throws IOException { - List regions = getMiniHBaseCluster().getRegions(tableName); - int totalRegionCount = regions.size(); - AtomicInteger prefetchedRegionCount = new AtomicInteger(); - for (HRegion r : regions) { - getMiniHBaseCluster().getClusterMetrics().getLiveServerMetrics().forEach((sn, sm) -> { - sm.getRegionMetrics().forEach((rn, rm) -> { - String regionNameAsString = r.getRegionInfo().getRegionNameAsString(); - String regionString = rm.getNameAsString(); - if (regionNameAsString.equals(regionString)) { - if (rm.getCurrentRegionCachedRatio() == 1.0f) { - prefetchedRegionCount.getAndIncrement(); - } - } - }); - }); - } - return getAdmin().tableExists(tableName) && totalRegionCount == prefetchedRegionCount.get(); - } - }; - } - - /** - * Returns a {@Link Predicate} for checking that at least one region for the table is prefetched - */ - public Waiter.Predicate predicateAtLeastOneRegionIsPrefetchedOnServer( - final TableName tableName, final ServerName serverName) { - return new ExplainingPredicate() { - @Override - public String explainFailure() throws IOException { - return "No Regions for table " + tableName.getNameAsString() + " prefetched on server " - + serverName.getAddress(); - } - - @Override - public boolean evaluate() throws IOException { - List regions = getMiniHBaseCluster().getRegions(tableName); - AtomicInteger prefetchedRegionCount = new AtomicInteger(); - ServerMetrics sm = - getMiniHBaseCluster().getClusterMetrics().getLiveServerMetrics().get(serverName); - for (HRegion r : regions) { - sm.getRegionMetrics().forEach((rn, rm) -> { - if ( - r.getRegionInfo().getRegionNameAsString().equals(rm.getNameAsString()) - && rm.getCurrentRegionCachedRatio() == 1.0f - ) { - prefetchedRegionCount.getAndIncrement(); - } - }); - } - return getAdmin().tableExists(tableName) && prefetchedRegionCount.get() > 0; - } - }; - } - - /** - * Returns a {@Link Predicate} for checking that more than half of the regions for the table are - * prefetched - */ - public Waiter.Predicate - predicateMajorityRegionsArePrefetched(final TableName tableName) { - return new ExplainingPredicate() { - @Override - public String explainFailure() throws IOException { - return "No Regions for table " + tableName.getNameAsString() + " prefetched"; - } - - @Override - public boolean evaluate() throws IOException { - List regions = getMiniHBaseCluster().getRegions(tableName); - int totalRegionCount = regions.size(); - AtomicInteger prefetchedRegionCount = new AtomicInteger(); - for (HRegion r : regions) { - getMiniHBaseCluster().getClusterMetrics().getLiveServerMetrics().forEach((sn, sm) -> { - sm.getRegionMetrics().forEach((rn, rm) -> { - String regionNameAsString = r.getRegionInfo().getRegionNameAsString(); - String regionString = rm.getNameAsString(); - if (regionNameAsString.equals(regionString)) { - if (rm.getCurrentRegionCachedRatio() == 1.0f) { - prefetchedRegionCount.getAndIncrement(); - } - } - }); - }); - } - return getAdmin().tableExists(tableName) - && (float) prefetchedRegionCount.get() / totalRegionCount > 0.5f; - } - }; - } - /** * Wait until no regions in transition. * @param timeout How long to wait. From 009c3b300a301eb4d8a45e83475c8261566c372e Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Tue, 5 Sep 2023 11:16:13 +0530 Subject: [PATCH 5/5] HBASE-27999 Implement cache prefetch aware load balancer --- .../master/balancer/BalancerClusterState.java | 147 ++++---- .../master/balancer/BalancerRegionLoad.java | 2 +- ...ancer.java => CacheAwareLoadBalancer.java} | 269 ++++++++------- .../balancer/StochasticLoadBalancer.java | 6 +- .../org/apache/hadoop/hbase/HConstants.java | 13 +- .../hbase/io/hfile/BlockCacheFactory.java | 13 +- ...r.java => TestCacheAwareLoadBalancer.java} | 108 +++--- ...stCacheAwareLoadBalancerCostFunctions.java | 316 +++++++++++++++++ ...refetchAwareLoadBalancerCostFunctions.java | 319 ------------------ 9 files changed, 596 insertions(+), 597 deletions(-) rename hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/{PrefetchAwareLoadBalancer.java => CacheAwareLoadBalancer.java} (57%) rename hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/{TestPrefetchAwareLoadBalancer.java => TestCacheAwareLoadBalancer.java} (77%) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancerCostFunctions.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancerCostFunctions.java 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 1078ade65545..4b3809c107cb 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 @@ -115,12 +115,12 @@ class BalancerClusterState { private float[][] rackLocalities; // Maps localityType -> region -> [server|rack]Index with highest locality private int[][] regionsToMostLocalEntities; - // Maps region -> serverIndex -> prefetchRatio of a region on a server - private Map, Float> regionIndexServerIndexPrefetchRatio; - // Maps regionIndex -> serverIndex with best prefetch ratio - private int[] regionServerIndexWithBestPrefetchRatio; - // Maps regionName -> oldServerName -> oldPrefetchRatio - Map> oldRegionServerPrefetchRatio; + // Maps region -> serverIndex -> regionCacheRatio of a region on a server + private Map, Float> regionIndexServerIndexRegionCachedRatio; + // Maps regionIndex -> serverIndex with best region cache ratio + private int[] regionServerIndexWithBestRegionCachedRatio; + // Maps regionName -> oldServerName -> cache ratio of the region on the old server + Map> regionCacheRatioOnOldServerMap; static class DefaultRackManager extends RackManager { @Override @@ -137,15 +137,15 @@ public String getRack(ServerName server) { protected BalancerClusterState(Map> clusterState, Map> loads, RegionHDFSBlockLocationFinder regionFinder, - RackManager rackManager, Map> oldRegionServerPrefetchRatio) { - this(null, clusterState, loads, regionFinder, rackManager, oldRegionServerPrefetchRatio); + RackManager rackManager, Map> oldRegionServerRegionCacheRatio) { + this(null, clusterState, loads, regionFinder, rackManager, oldRegionServerRegionCacheRatio); } @SuppressWarnings("unchecked") BalancerClusterState(Collection unassignedRegions, Map> clusterState, Map> loads, RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager, - Map> oldRegionServerPrefetchRatio) { + Map> oldRegionServerRegionCacheRatio) { if (unassignedRegions == null) { unassignedRegions = Collections.emptyList(); } @@ -159,7 +159,7 @@ protected BalancerClusterState(Map> clusterState, tables = new ArrayList<>(); this.rackManager = rackManager != null ? rackManager : new DefaultRackManager(); - this.oldRegionServerPrefetchRatio = oldRegionServerPrefetchRatio; + this.regionCacheRatioOnOldServerMap = oldRegionServerRegionCacheRatio; numRegions = 0; @@ -570,21 +570,21 @@ public int getTotalRegionHFileSizeMB(int region) { } /** - * Returns the weighted prefetch ratio of a region on the given region server + * Returns the weighted cache ratio of a region on the given region server */ - public float getOrComputeWeightedPrefetchRatio(int region, int server) { - return getTotalRegionHFileSizeMB(region) * getOrComputeRegionPrefetchRatio(region, server); + public float getOrComputeWeightedRegionCacheRatio(int region, int server) { + return getTotalRegionHFileSizeMB(region) * getOrComputeRegionCacheRatio(region, server); } /** - * Returns the amount by which a region is prefetched on a given region server. If the region is - * not currently hosted on the given region server, then find out if it was previously hosted - * there and return the old prefetch ratio. + * Returns the amount by which a region is cached on a given region server. If the region is not + * currently hosted on the given region server, then find out if it was previously hosted there + * and return the old cache ratio. */ - protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) { - float prefetchRatio = 0.0f; + protected float getRegionCacheRatioOnRegionServer(int region, int regionServerIndex) { + float regionCacheRatio = 0.0f; - // Get the current prefetch ratio if the region is hosted on the server regionServerIndex + // Get the current region cache ratio if the region is hosted on the server regionServerIndex for (int regionIndex : regionsPerServer[regionServerIndex]) { if (region != regionIndex) { continue; @@ -592,104 +592,105 @@ protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) Deque regionLoadList = regionLoads[regionIndex]; - // The region is currently hosted on this region server. Get the prefetch ratio for this + // The region is currently hosted on this region server. Get the region cache ratio for this // region on this server - prefetchRatio = - regionLoadList == null ? 0.0f : regionLoadList.getLast().getCurrentRegionPrefetchRatio(); + regionCacheRatio = + regionLoadList == null ? 0.0f : regionLoadList.getLast().getCurrentRegionCacheRatio(); - return prefetchRatio; + return regionCacheRatio; } - // Region is not currently hosted on this server. Check if the region was prefetched on this + // Region is not currently hosted on this server. Check if the region was cached on this // server earlier. This can happen when the server was shutdown and the cache was persisted. // Search using the region name and server name and not the index id and server id as these ids // may change when a server is marked as dead or a new server is added. String regionEncodedName = regions[region].getEncodedName(); ServerName serverName = servers[regionServerIndex]; if ( - oldRegionServerPrefetchRatio != null - && oldRegionServerPrefetchRatio.containsKey(regionEncodedName) + regionCacheRatioOnOldServerMap != null + && regionCacheRatioOnOldServerMap.containsKey(regionEncodedName) ) { - Pair serverPrefetchRatio = - oldRegionServerPrefetchRatio.get(regionEncodedName); - if (ServerName.isSameAddress(serverPrefetchRatio.getFirst(), serverName)) { - prefetchRatio = serverPrefetchRatio.getSecond(); + Pair cacheRatioOfRegionOnServer = + regionCacheRatioOnOldServerMap.get(regionEncodedName); + if (ServerName.isSameAddress(cacheRatioOfRegionOnServer.getFirst(), serverName)) { + regionCacheRatio = cacheRatioOfRegionOnServer.getSecond(); if (LOG.isDebugEnabled()) { - LOG.debug("Old prefetch ratio found for region {} on server {}: {}", regionEncodedName, - serverName, prefetchRatio); + LOG.debug("Old cache ratio found for region {} on server {}: {}", regionEncodedName, + serverName, regionCacheRatio); } } } - return prefetchRatio; + return regionCacheRatio; } /** - * Populate the maps containing information about how much a region is prefetched on a region - * server. + * Populate the maps containing information about how much a region is cached on a region server. */ - private void computeRegionServerPrefetchRatio() { - regionIndexServerIndexPrefetchRatio = new HashMap<>(); - regionServerIndexWithBestPrefetchRatio = new int[numRegions]; + private void computeRegionServerRegionCacheRatio() { + regionIndexServerIndexRegionCachedRatio = new HashMap<>(); + regionServerIndexWithBestRegionCachedRatio = new int[numRegions]; for (int region = 0; region < numRegions; region++) { - float bestPrefetchRatio = 0.0f; - int serverWithBestPrefetchRatio = 0; + float bestRegionCacheRatio = 0.0f; + int serverWithBestRegionCacheRatio = 0; for (int server = 0; server < numServers; server++) { - float prefetchRatio = getRegionServerPrefetchRatio(region, server); - if (prefetchRatio > 0.0f || server == regionIndexToServerIndex[region]) { - // A region with prefetch ratio 0 on a server means nothing. Hence, just make a note of - // prefetch only if the prefetch ratio is greater than 0. + float regionCacheRatio = getRegionCacheRatioOnRegionServer(region, server); + if (regionCacheRatio > 0.0f || server == regionIndexToServerIndex[region]) { + // A region with cache ratio 0 on a server means nothing. Hence, just make a note of + // cache ratio only if the cache ratio is greater than 0. Pair regionServerPair = new Pair<>(region, server); - regionIndexServerIndexPrefetchRatio.put(regionServerPair, prefetchRatio); + regionIndexServerIndexRegionCachedRatio.put(regionServerPair, regionCacheRatio); } - if (prefetchRatio > bestPrefetchRatio) { - serverWithBestPrefetchRatio = server; - // If the server currently hosting the region has equal prefetch ratio to a historical + if (regionCacheRatio > bestRegionCacheRatio) { + serverWithBestRegionCacheRatio = server; + // If the server currently hosting the region has equal cache ratio to a historical // server, consider the current server to keep hosting the region - bestPrefetchRatio = prefetchRatio; - } else - if (prefetchRatio == bestPrefetchRatio && server == regionIndexToServerIndex[region]) { - // If two servers have same prefetch ratio, then the server currently hosting the region - // should retain the region - serverWithBestPrefetchRatio = server; - } - } - regionServerIndexWithBestPrefetchRatio[region] = serverWithBestPrefetchRatio; + bestRegionCacheRatio = regionCacheRatio; + } else if ( + regionCacheRatio == bestRegionCacheRatio && server == regionIndexToServerIndex[region] + ) { + // If two servers have same region cache ratio, then the server currently hosting the + // region + // should retain the region + serverWithBestRegionCacheRatio = server; + } + } + regionServerIndexWithBestRegionCachedRatio[region] = serverWithBestRegionCacheRatio; Pair regionServerPair = new Pair<>(region, regionIndexToServerIndex[region]); - float tempPrefetchRatio = regionIndexServerIndexPrefetchRatio.get(regionServerPair); - if (tempPrefetchRatio > bestPrefetchRatio) { + float tempRegionCacheRatio = regionIndexServerIndexRegionCachedRatio.get(regionServerPair); + if (tempRegionCacheRatio > bestRegionCacheRatio) { LOG.warn( - "INVALID CONDITION: region {} on server {} prefetch ratio {} is greater than the " - + "best prefetch ratio {} on server {}", + "INVALID CONDITION: region {} on server {} cache ratio {} is greater than the " + + "best region cache ratio {} on server {}", regions[region].getEncodedName(), servers[regionIndexToServerIndex[region]], - tempPrefetchRatio, bestPrefetchRatio, servers[serverWithBestPrefetchRatio]); + tempRegionCacheRatio, bestRegionCacheRatio, servers[serverWithBestRegionCacheRatio]); } } } - protected float getOrComputeRegionPrefetchRatio(int region, int server) { + protected float getOrComputeRegionCacheRatio(int region, int server) { if ( - regionServerIndexWithBestPrefetchRatio == null - || regionIndexServerIndexPrefetchRatio.isEmpty() + regionServerIndexWithBestRegionCachedRatio == null + || regionIndexServerIndexRegionCachedRatio.isEmpty() ) { - computeRegionServerPrefetchRatio(); + computeRegionServerRegionCacheRatio(); } Pair regionServerPair = new Pair<>(region, server); - return regionIndexServerIndexPrefetchRatio.containsKey(regionServerPair) - ? regionIndexServerIndexPrefetchRatio.get(regionServerPair) + return regionIndexServerIndexRegionCachedRatio.containsKey(regionServerPair) + ? regionIndexServerIndexRegionCachedRatio.get(regionServerPair) : 0.0f; } - public int[] getOrComputeServerWithBestPrefetchRatio() { + public int[] getOrComputeServerWithBestRegionCachedRatio() { if ( - regionServerIndexWithBestPrefetchRatio == null - || regionIndexServerIndexPrefetchRatio.isEmpty() + regionServerIndexWithBestRegionCachedRatio == null + || regionIndexServerIndexRegionCachedRatio.isEmpty() ) { - computeRegionServerPrefetchRatio(); + computeRegionServerRegionCacheRatio(); } - return regionServerIndexWithBestPrefetchRatio; + return regionServerIndexWithBestRegionCachedRatio; } /** diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java index ad6cc1e0e9a6..33d00e3de862 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerRegionLoad.java @@ -71,7 +71,7 @@ public int getRegionSizeMB() { return regionSizeMB; } - public float getCurrentRegionPrefetchRatio() { + public float getCurrentRegionCacheRatio() { return currentRegionPrefetchRatio; } } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CacheAwareLoadBalancer.java similarity index 57% rename from hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java rename to hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CacheAwareLoadBalancer.java index 632f006259ba..d73769a3971b 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchAwareLoadBalancer.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/CacheAwareLoadBalancer.java @@ -18,14 +18,16 @@ package org.apache.hadoop.hbase.master.balancer; /** An implementation of the {@link org.apache.hadoop.hbase.master.LoadBalancer} that assigns regions - * based on the amount they are prefetched on a given server. A region can move across the region + * based on the amount they are cached on a given server. A region can move across the region * servers whenever a region server shuts down or crashes. The region server preserves the cache * periodically and restores the cache when it is restarted. This balancer implements a mechanism - * where it maintains the amount by which a region is prefetched on a region server. During balancer - * run, a region plan is generated that takes into account this prefetch information and tries to + * where it maintains the amount by which a region is cached on a region server. During balancer + * run, a region plan is generated that takes into account this cache information and tries to * move the regions so that the cache minimally impacted. */ +import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_PERSISTENT_PATH_KEY; + import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; @@ -36,7 +38,6 @@ import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.RegionMetrics; import org.apache.hadoop.hbase.ServerMetrics; import org.apache.hadoop.hbase.ServerName; @@ -48,14 +49,14 @@ import org.slf4j.LoggerFactory; @InterfaceAudience.Private -public class PrefetchAwareLoadBalancer extends StochasticLoadBalancer { - private static final Logger LOG = LoggerFactory.getLogger(PrefetchAwareLoadBalancer.class); +public class CacheAwareLoadBalancer extends StochasticLoadBalancer { + private static final Logger LOG = LoggerFactory.getLogger(CacheAwareLoadBalancer.class); private Configuration configuration; public enum GeneratorFunctionType { LOAD, - PREFETCH + CACHE_RATIO } @Override @@ -69,17 +70,17 @@ public synchronized void loadConf(Configuration configuration) { protected List createCandidateGenerators() { List candidateGenerators = new ArrayList<>(2); candidateGenerators.add(GeneratorFunctionType.LOAD.ordinal(), - new PrefetchBasedSkewnessCandidateGenerator()); - candidateGenerators.add(GeneratorFunctionType.PREFETCH.ordinal(), - new PrefetchCandidateGenerator()); + new CacheAwareSkewnessCandidateGenerator()); + candidateGenerators.add(GeneratorFunctionType.CACHE_RATIO.ordinal(), + new CacheAwareCandidateGenerator()); return candidateGenerators; } @Override protected List createCostFunctions(Configuration configuration) { List costFunctions = new ArrayList<>(); - addCostFunction(costFunctions, new PrefetchAwareRegionSkewnessCostFunction(configuration)); - addCostFunction(costFunctions, new PrefetchCacheCostFunction(configuration)); + addCostFunction(costFunctions, new CacheAwareRegionSkewnessCostFunction(configuration)); + addCostFunction(costFunctions, new CacheAwareCostFunction(configuration)); return costFunctions; } @@ -96,14 +97,14 @@ public void updateClusterMetrics(ClusterMetrics clusterMetrics) { } /** - * Collect the prefetch information for all the regions from all the active region servers. + * Collect the amount of region cached for all the regions from all the active region servers. */ private void updateRegionLoad() { loads = new HashMap<>(); - oldRegionPrefetchMap = new HashMap<>(); - Map> currentPrefetchMap = new HashMap<>(); + regionCacheRatioOnOldServerMap = new HashMap<>(); + Map> regionCacheRatioOnCurrentServerMap = new HashMap<>(); - // Build current region prefetch statistics + // Build current region cache statistics clusterStatus.getLiveServerMetrics().forEach((ServerName sn, ServerMetrics sm) -> { // Create a map of region and the server where it is currently hosted sm.getRegionMetrics().forEach((byte[] regionName, RegionMetrics rm) -> { @@ -115,29 +116,31 @@ private void updateRegionLoad() { int regionSizeMB = (int) rm.getRegionSizeMB().get(Size.Unit.MEGABYTE); rload.add(new BalancerRegionLoad(rm)); - // Maintain a map of region and it's total size. This is needed to calculate the prefetch + // Maintain a map of region and it's total size. This is needed to calculate the cache // ratios for the regions cached on old region servers - currentPrefetchMap.put(regionEncodedName, new Pair<>(sn, regionSizeMB)); + regionCacheRatioOnCurrentServerMap.put(regionEncodedName, new Pair<>(sn, regionSizeMB)); loads.put(regionEncodedName, rload); }); }); - // Build olf region prefetch statistics + // Build cache statistics for the regions hosted previously on old region servers clusterStatus.getLiveServerMetrics().forEach((ServerName sn, ServerMetrics sm) -> { // Find if a region was previously hosted on a server other than the one it is currently // hosted on. - sm.getRegionCachedInfo().forEach((String regionEncodedName, Integer prefetchSize) -> { - // If the region is fond in currentPrefetchMap, it is currently hosted on this server - if (currentPrefetchMap.containsKey(regionEncodedName)) { - ServerName currentServer = currentPrefetchMap.get(regionEncodedName).getFirst(); + sm.getRegionCachedInfo().forEach((String regionEncodedName, Integer regionSizeInCache) -> { + // If the region is found in regionCacheRatioOnCurrentServerMap, it is currently hosted on + // this server + if (regionCacheRatioOnCurrentServerMap.containsKey(regionEncodedName)) { + ServerName currentServer = + regionCacheRatioOnCurrentServerMap.get(regionEncodedName).getFirst(); if (!ServerName.isSameAddress(currentServer, sn)) { - int regionSizeMB = currentPrefetchMap.get(regionEncodedName).getSecond(); - float regionPrefetchRatio = - regionSizeMB == 0 ? 0.0f : (float) prefetchSize / regionSizeMB; - oldRegionPrefetchMap.put(regionEncodedName, new Pair<>(sn, regionPrefetchRatio)); + int regionSizeMB = + regionCacheRatioOnCurrentServerMap.get(regionEncodedName).getSecond(); + float regionCacheRatioOnOldServer = + regionSizeMB == 0 ? 0.0f : (float) regionSizeInCache / regionSizeMB; + regionCacheRatioOnOldServerMap.put(regionEncodedName, + new Pair<>(sn, regionCacheRatioOnOldServer)); } - } else { - LOG.warn("Region {} not currently hosted on any server", regionEncodedName); } }); }); @@ -155,133 +158,142 @@ private RegionInfo getRegionInfoByEncodedName(BalancerClusterState cluster, Stri return null; } - private class PrefetchCandidateGenerator extends CandidateGenerator { + private class CacheAwareCandidateGenerator extends CandidateGenerator { @Override protected BalanceAction generate(BalancerClusterState cluster) { - // Move the regions to the servers they were previously hosted on based on the prefetch ratio - if (!oldRegionPrefetchMap.isEmpty() && oldRegionPrefetchMap.entrySet().iterator().hasNext()) { - Map.Entry> regionPrefetchEntry = - oldRegionPrefetchMap.entrySet().iterator().next(); + // Move the regions to the servers they were previously hosted on based on the cache ratio + if ( + !regionCacheRatioOnOldServerMap.isEmpty() + && regionCacheRatioOnOldServerMap.entrySet().iterator().hasNext() + ) { + Map.Entry> regionCacheRatioServerMap = + regionCacheRatioOnOldServerMap.entrySet().iterator().next(); // Get the server where this region was previously hosted - String regionEncodedName = regionPrefetchEntry.getKey(); + String regionEncodedName = regionCacheRatioServerMap.getKey(); RegionInfo regionInfo = getRegionInfoByEncodedName(cluster, regionEncodedName); if (regionInfo == null) { LOG.warn("Region {} not found", regionEncodedName); - oldRegionPrefetchMap.remove(regionEncodedName); + regionCacheRatioOnOldServerMap.remove(regionEncodedName); return BalanceAction.NULL_ACTION; } if (regionInfo.isMetaRegion() || regionInfo.getTable().isSystemTable()) { - oldRegionPrefetchMap.remove(regionEncodedName); + regionCacheRatioOnOldServerMap.remove(regionEncodedName); return BalanceAction.NULL_ACTION; } int regionIndex = cluster.regionsToIndex.get(regionInfo); int oldServerIndex = cluster.serversToIndex - .get(oldRegionPrefetchMap.get(regionEncodedName).getFirst().getAddress()); + .get(regionCacheRatioOnOldServerMap.get(regionEncodedName).getFirst().getAddress()); if (oldServerIndex < 0) { LOG.warn("Server previously hosting region {} not found", regionEncodedName); - oldRegionPrefetchMap.remove(regionEncodedName); + regionCacheRatioOnOldServerMap.remove(regionEncodedName); return BalanceAction.NULL_ACTION; } - float oldPrefetchRatio = - cluster.getOrComputeRegionPrefetchRatio(regionIndex, oldServerIndex); + float oldRegionCacheRatio = + cluster.getOrComputeRegionCacheRatio(regionIndex, oldServerIndex); int currentServerIndex = cluster.regionIndexToServerIndex[regionIndex]; - float currentPrefetchRatio = - cluster.getOrComputeRegionPrefetchRatio(regionIndex, currentServerIndex); + float currentRegionCacheRatio = + cluster.getOrComputeRegionCacheRatio(regionIndex, currentServerIndex); BalanceAction action = generatePlan(cluster, regionIndex, currentServerIndex, - currentPrefetchRatio, oldServerIndex, oldPrefetchRatio); - oldRegionPrefetchMap.remove(regionEncodedName); + currentRegionCacheRatio, oldServerIndex, oldRegionCacheRatio); + regionCacheRatioOnOldServerMap.remove(regionEncodedName); return action; } return BalanceAction.NULL_ACTION; } private BalanceAction generatePlan(BalancerClusterState cluster, int regionIndex, - int currentServerIndex, float currentServerPrefetchRatio, int oldServerIndex, - float oldServerPrefetchRatio) { + int currentServerIndex, float cacheRatioOnCurrentServer, int oldServerIndex, + float cacheRatioOnOldServer) { return moveRegionToOldServer(cluster, regionIndex, currentServerIndex, - currentServerPrefetchRatio, oldServerIndex, oldServerPrefetchRatio) + cacheRatioOnCurrentServer, oldServerIndex, cacheRatioOnOldServer) ? getAction(currentServerIndex, regionIndex, oldServerIndex, -1) : BalanceAction.NULL_ACTION; } private boolean moveRegionToOldServer(BalancerClusterState cluster, int regionIndex, - int currentServerIndex, float currentServerPrefetchRatio, int oldServerIndex, - float oldServerPrefetchRatio) { + int currentServerIndex, float cacheRatioOnCurrentServer, int oldServerIndex, + float cacheRatioOnOldServer) { // Find if the region has already moved by comparing the current server index with the // current server index. This can happen when other candidate generator has moved the region if (currentServerIndex < 0 || oldServerIndex < 0) { return false; } - float prefetchDiffThreshold = 0.6f; + float cacheRatioDiffThreshold = 0.6f; // Conditions for moving the region - // If the region is fully prefetched on the old server, move the region back - if (oldServerPrefetchRatio == 1.0f) { + // If the region is fully cached on the old server, move the region back + if (cacheRatioOnOldServer == 1.0f) { if (LOG.isDebugEnabled()) { - LOG.debug("Region {} moved to the old server {} as it is fully prefetched there", + LOG.debug("Region {} moved to the old server {} as it is fully cached there", cluster.regions[regionIndex].getEncodedName(), cluster.servers[oldServerIndex]); } return true; } - // Move the region back to the old server if it is prefetched equally on both the servers - if (currentServerPrefetchRatio == oldServerPrefetchRatio) { + // Move the region back to the old server if it is cached equally on both the servers + if (cacheRatioOnCurrentServer == cacheRatioOnOldServer) { if (LOG.isDebugEnabled()) { - LOG.debug("Region {} moved from {} to {} as the prefetch {} is same on both servers", + LOG.debug( + "Region {} moved from {} to {} as the region is cached {} equally on both servers", cluster.regions[regionIndex].getEncodedName(), cluster.servers[currentServerIndex], - cluster.servers[oldServerIndex], currentServerPrefetchRatio); + cluster.servers[oldServerIndex], cacheRatioOnCurrentServer); } return true; } - // If the region is not fully prefetched on either of the servers, move the region back to the - // old server if the current server prefetch is still much less than the old server + // If the region is not fully cached on either of the servers, move the region back to the + // old server if the region cache ratio on the current server is still much less than the old + // server if ( - oldServerPrefetchRatio > 0.0f - && currentServerPrefetchRatio / oldServerPrefetchRatio < prefetchDiffThreshold + cacheRatioOnOldServer > 0.0f + && cacheRatioOnCurrentServer / cacheRatioOnOldServer < cacheRatioDiffThreshold ) { if (LOG.isDebugEnabled()) { LOG.debug( - "Region {} moved from {} to {} old prefetch {} is better than the current " - + "prefetch {}", + "Region {} moved from {} to {} as region cache ratio {} is better than the current " + + "cache ratio {}", cluster.regions[regionIndex].getEncodedName(), cluster.servers[currentServerIndex], - cluster.servers[oldServerIndex], currentServerPrefetchRatio, oldServerPrefetchRatio); + cluster.servers[oldServerIndex], cacheRatioOnCurrentServer, cacheRatioOnOldServer); } return true; } if (LOG.isDebugEnabled()) { - LOG.debug("Region {} not moved from {} to {} with current prefetch {} and old prefetch {}", + LOG.debug( + "Region {} not moved from {} to {} with current cache ratio {} and old cache ratio {}", cluster.regions[regionIndex], cluster.servers[currentServerIndex], - cluster.servers[oldServerIndex], currentServerPrefetchRatio, oldServerPrefetchRatio); + cluster.servers[oldServerIndex], cacheRatioOnCurrentServer, cacheRatioOnOldServer); } return false; } } - private class PrefetchBasedSkewnessCandidateGenerator extends LoadCandidateGenerator { + private class CacheAwareSkewnessCandidateGenerator extends LoadCandidateGenerator { @Override BalanceAction pickRandomRegions(BalancerClusterState cluster, int thisServer, int otherServer) { // First move all the regions which were hosted previously on some other server back to their // old servers - if (!oldRegionPrefetchMap.isEmpty() && oldRegionPrefetchMap.entrySet().iterator().hasNext()) { - // Get the first region index in the historical prefetch list + if ( + !regionCacheRatioOnOldServerMap.isEmpty() + && regionCacheRatioOnOldServerMap.entrySet().iterator().hasNext() + ) { + // Get the first region index in the historical cache ratio list Map.Entry> regionEntry = - oldRegionPrefetchMap.entrySet().iterator().next(); + regionCacheRatioOnOldServerMap.entrySet().iterator().next(); String regionEncodedName = regionEntry.getKey(); RegionInfo regionInfo = getRegionInfoByEncodedName(cluster, regionEncodedName); if (regionInfo == null) { LOG.warn("Region {} does not exist", regionEncodedName); - oldRegionPrefetchMap.remove(regionEncodedName); + regionCacheRatioOnOldServerMap.remove(regionEncodedName); return BalanceAction.NULL_ACTION; } if (regionInfo.isMetaRegion() || regionInfo.getTable().isSystemTable()) { - oldRegionPrefetchMap.remove(regionEncodedName); + regionCacheRatioOnOldServerMap.remove(regionEncodedName); return BalanceAction.NULL_ACTION; } @@ -293,14 +305,14 @@ BalanceAction pickRandomRegions(BalancerClusterState cluster, int thisServer, in // Get the old server index otherServer = cluster.serversToIndex.get(regionEntry.getValue().getFirst().getAddress()); - oldRegionPrefetchMap.remove(regionEncodedName); + regionCacheRatioOnOldServerMap.remove(regionEncodedName); if (otherServer < 0) { // The old server has been moved to other host and hence, the region cannot be moved back // to the old server if (LOG.isDebugEnabled()) { LOG.debug( - "PrefetchBasedSkewnessCandidateGenerator: Region {} not moved to the old " + "CacheAwareSkewnessCandidateGenerator: Region {} not moved to the old " + "server {} as the server does not exist", regionEncodedName, regionEntry.getValue().getFirst().getHostname()); } @@ -309,7 +321,7 @@ BalanceAction pickRandomRegions(BalancerClusterState cluster, int thisServer, in if (LOG.isDebugEnabled()) { LOG.debug( - "PrefetchBasedSkewnessCandidateGenerator: Region {} moved from {} to {} as it " + "CacheAwareSkewnessCandidateGenerator: Region {} moved from {} to {} as it " + "was hosted their earlier", regionEncodedName, cluster.servers[thisServer].getHostname(), cluster.servers[otherServer].getHostname()); @@ -322,47 +334,47 @@ BalanceAction pickRandomRegions(BalancerClusterState cluster, int thisServer, in return BalanceAction.NULL_ACTION; } - int regionIndexToMove = pickLeastPrefetchedRegion(cluster, thisServer); + int regionIndexToMove = pickLeastCachedRegion(cluster, thisServer); if (regionIndexToMove < 0) { if (LOG.isDebugEnabled()) { - LOG.debug("PrefetchBasedSkewnessCandidateGenerator: No region found for movement"); + LOG.debug("CacheAwareSkewnessCandidateGenerator: No region found for movement"); } return BalanceAction.NULL_ACTION; } if (LOG.isDebugEnabled()) { LOG.debug( - "PrefetchBasedSkewnessCandidateGenerator: Region {} moved from {} to {} as it is " - + "least prefetched on current server", + "CacheAwareSkewnessCandidateGenerator: Region {} moved from {} to {} as it is " + + "least cached on current server", cluster.regions[regionIndexToMove].getEncodedName(), cluster.servers[thisServer].getHostname(), cluster.servers[otherServer].getHostname()); } return getAction(thisServer, regionIndexToMove, otherServer, -1); } - private int pickLeastPrefetchedRegion(BalancerClusterState cluster, int thisServer) { - float minPrefetchRatio = Float.MAX_VALUE; - int regionWithMinPrefetch = -1; + private int pickLeastCachedRegion(BalancerClusterState cluster, int thisServer) { + float minCacheRatio = Float.MAX_VALUE; + int leastCachedRegion = -1; for (int i = 0; i < cluster.regionsPerServer[thisServer].length; i++) { int regionIndex = cluster.regionsPerServer[thisServer][i]; - float currentPrefetchRatio = - cluster.getOrComputeRegionPrefetchRatio(regionIndex, thisServer); - if (currentPrefetchRatio < minPrefetchRatio) { - minPrefetchRatio = currentPrefetchRatio; - regionWithMinPrefetch = regionIndex; + float cacheRatioOnCurrentServer = + cluster.getOrComputeRegionCacheRatio(regionIndex, thisServer); + if (cacheRatioOnCurrentServer < minCacheRatio) { + minCacheRatio = cacheRatioOnCurrentServer; + leastCachedRegion = regionIndex; } } - return regionWithMinPrefetch; + return leastCachedRegion; } } - static class PrefetchAwareRegionSkewnessCostFunction extends CostFunction { + static class CacheAwareRegionSkewnessCostFunction extends CostFunction { static final String REGION_COUNT_SKEW_COST_KEY = "hbase.master.balancer.stochastic.regionCountCost"; static final float DEFAULT_REGION_COUNT_SKEW_COST = 20; private final DoubleArrayCost cost = new DoubleArrayCost(); - PrefetchAwareRegionSkewnessCostFunction(Configuration conf) { + CacheAwareRegionSkewnessCostFunction(Configuration conf) { // Load multiplier should be the greatest as it is the most general way to balance data. this.setMultiplier(conf.getFloat(REGION_COUNT_SKEW_COST_KEY, DEFAULT_REGION_COUNT_SKEW_COST)); } @@ -396,73 +408,72 @@ public final void updateWeight(double[] weights) { } } - static class PrefetchCacheCostFunction extends CostFunction { - private static final String PREFETCH_CACHE_COST_KEY = - "hbase.master.balancer.stochastic.prefetchCacheCost"; - private String prefetchedFileListPath; - private double prefetchRatio; - private double bestPrefetchRatio; - - private static final float DEFAULT_PREFETCH_COST = 20; - - PrefetchCacheCostFunction(Configuration conf) { - prefetchedFileListPath = conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY); - // Disable the prefetchCacheCostFunction if the prefetch file list persistence is not enabled - this.setMultiplier(prefetchedFileListPath == null - ? 0.0f - : conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_COST)); - bestPrefetchRatio = 0.0; - prefetchRatio = 0.0; + static class CacheAwareCostFunction extends CostFunction { + private static final String CACHE_COST_KEY = "hbase.master.balancer.stochastic.cacheCost"; + private double cacheRatio; + private double bestCacheRatio; + + private static final float DEFAULT_CACHE_COST = 20; + + CacheAwareCostFunction(Configuration conf) { + boolean isPersistentCache = conf.get(BUCKET_CACHE_PERSISTENT_PATH_KEY) != null; + // Disable the CacheAwareCostFunction if the cached file list persistence is not enabled + this.setMultiplier( + !isPersistentCache ? 0.0f : conf.getFloat(CACHE_COST_KEY, DEFAULT_CACHE_COST)); + bestCacheRatio = 0.0; + cacheRatio = 0.0; } @Override void prepare(BalancerClusterState cluster) { super.prepare(cluster); - prefetchRatio = 0.0; - bestPrefetchRatio = 0.0; + cacheRatio = 0.0; + bestCacheRatio = 0.0; for (int region = 0; region < cluster.numRegions; region++) { - prefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region, + cacheRatio += cluster.getOrComputeWeightedRegionCacheRatio(region, cluster.regionIndexToServerIndex[region]); - bestPrefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region, - getServerWithBestPrefetchRatioForRegion(region)); + bestCacheRatio += cluster.getOrComputeWeightedRegionCacheRatio(region, + getServerWithBestCacheRatioForRegion(region)); } - prefetchRatio = bestPrefetchRatio == 0 ? 1.0 : prefetchRatio / bestPrefetchRatio; + cacheRatio = bestCacheRatio == 0 ? 1.0 : cacheRatio / bestCacheRatio; if (LOG.isDebugEnabled()) { - LOG.debug("PrefetchCacheCostFunction: Cost: {}", 1 - prefetchRatio); + LOG.debug("CacheAwareCostFunction: Cost: {}", 1 - cacheRatio); } } @Override protected double cost() { - return scale(0, 1, 1 - prefetchRatio); + return scale(0, 1, 1 - cacheRatio); } @Override protected void regionMoved(int region, int oldServer, int newServer) { - double oldServerPrefetch = cluster.getOrComputeWeightedPrefetchRatio(region, oldServer); - double newServerPrefetch = cluster.getOrComputeWeightedPrefetchRatio(region, newServer); - double prefetchDelta = newServerPrefetch - oldServerPrefetch; - double normalizedDelta = bestPrefetchRatio == 0.0 ? 0.0 : prefetchDelta / bestPrefetchRatio; - prefetchRatio += normalizedDelta; - if (LOG.isDebugEnabled() && (prefetchRatio < 0.0 || prefetchRatio > 1.0)) { + double regionCacheRatioOnOldServer = + cluster.getOrComputeWeightedRegionCacheRatio(region, oldServer); + double regionCacheRatioOnNewServer = + cluster.getOrComputeWeightedRegionCacheRatio(region, newServer); + double cacheRatioDiff = regionCacheRatioOnNewServer - regionCacheRatioOnOldServer; + double normalizedDelta = bestCacheRatio == 0.0 ? 0.0 : cacheRatioDiff / bestCacheRatio; + cacheRatio += normalizedDelta; + if (LOG.isDebugEnabled() && (cacheRatio < 0.0 || cacheRatio > 1.0)) { LOG.debug( - "PrefetchCacheCostFunction:regionMoved:region:{}:from:{}:to:{}:oldServerPrefetch:{}:" - + "newServerPrefetch:{}:bestPrefetchRatio:{}:prefetchRatio:{}", + "CacheAwareCostFunction:regionMoved:region:{}:from:{}:to:{}:regionCacheRatioOnOldServer:{}:" + + "regionCacheRatioOnNewServer:{}:bestRegionCacheRatio:{}:cacheRatio:{}", cluster.regions[region].getEncodedName(), cluster.servers[oldServer].getHostname(), - cluster.servers[newServer].getHostname(), oldServerPrefetch, newServerPrefetch, - bestPrefetchRatio, prefetchRatio); + cluster.servers[newServer].getHostname(), regionCacheRatioOnOldServer, + regionCacheRatioOnNewServer, bestCacheRatio, cacheRatio); } } - private int getServerWithBestPrefetchRatioForRegion(int region) { - return cluster.getOrComputeServerWithBestPrefetchRatio()[region]; + private int getServerWithBestCacheRatioForRegion(int region) { + return cluster.getOrComputeServerWithBestRegionCachedRatio()[region]; } @Override public final void updateWeight(double[] weights) { - weights[GeneratorFunctionType.PREFETCH.ordinal()] += cost(); + weights[GeneratorFunctionType.CACHE_RATIO.ordinal()] += cost(); } } } 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 8316300958e2..e5cd5446c5c8 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 @@ -137,7 +137,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { private long maxRunningTime = DEFAULT_MAX_RUNNING_TIME; private int numRegionLoadsToRemember = DEFAULT_KEEP_REGION_LOADS; private float minCostNeedBalance = DEFAULT_MIN_COST_NEED_BALANCE; - Map> oldRegionPrefetchMap = new HashMap<>(); + Map> regionCacheRatioOnOldServerMap = new HashMap<>(); protected List costFunctions; // FindBugs: Wants this protected; // IS2_INCONSISTENT_SYNC @@ -467,8 +467,8 @@ protected List balanceTable(TableName tableName, // The clusterState that is given to this method contains the state // of all the regions in the table(s) (that's true today) // Keep track of servers to iterate through them. - BalancerClusterState cluster = - new BalancerClusterState(loadOfOneTable, loads, finder, rackManager, oldRegionPrefetchMap); + BalancerClusterState cluster = new BalancerClusterState(loadOfOneTable, loads, finder, + rackManager, regionCacheRatioOnOldServerMap); long startTime = EnvironmentEdgeManager.currentTime(); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 5ae04048ddbd..2aa9ecf69ec4 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1336,9 +1336,16 @@ public enum OperationStatusCode { public static final String BUCKET_CACHE_SIZE_KEY = "hbase.bucketcache.size"; /** - * Prefetch persistence path key - */ - public static final String PREFETCH_PERSISTENCE_PATH_KEY = "hbase.prefetch.file.list.path"; + * If the chosen ioengine can persist its state across restarts, the path to the file to persist + * to. This file is NOT the data file. It is a file into which we will serialize the map of what + * is in the data file. For example, if you pass the following argument as + * BUCKET_CACHE_IOENGINE_KEY ("hbase.bucketcache.ioengine"), + * file:/tmp/bucketcache.data , then we will write the bucketcache data to the file + * /tmp/bucketcache.data but the metadata on where the data is in the supplied file + * is an in-memory map that needs to be persisted across restarts. Where to store this in-memory + * state is what you supply here: e.g. /tmp/bucketcache.map. + */ + public static final String BUCKET_CACHE_PERSISTENT_PATH_KEY = "hbase.bucketcache.persistent.path"; /** * HConstants for fast fail on the client side follow diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java index 38a296aad523..6956d584d92a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.io.hfile; import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_IOENGINE_KEY; +import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_PERSISTENT_PATH_KEY; import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_SIZE_KEY; import java.io.IOException; @@ -47,18 +48,6 @@ public final class BlockCacheFactory { public static final String BLOCKCACHE_POLICY_KEY = "hfile.block.cache.policy"; public static final String BLOCKCACHE_POLICY_DEFAULT = "LRU"; - /** - * If the chosen ioengine can persist its state across restarts, the path to the file to persist - * to. This file is NOT the data file. It is a file into which we will serialize the map of what - * is in the data file. For example, if you pass the following argument as - * BUCKET_CACHE_IOENGINE_KEY ("hbase.bucketcache.ioengine"), - * file:/tmp/bucketcache.data , then we will write the bucketcache data to the file - * /tmp/bucketcache.data but the metadata on where the data is in the supplied file - * is an in-memory map that needs to be persisted across restarts. Where to store this in-memory - * state is what you supply here: e.g. /tmp/bucketcache.map. - */ - public static final String BUCKET_CACHE_PERSISTENT_PATH_KEY = "hbase.bucketcache.persistent.path"; - public static final String BUCKET_CACHE_WRITER_THREADS_KEY = "hbase.bucketcache.writer.threads"; public static final String BUCKET_CACHE_WRITER_QUEUE_KEY = "hbase.bucketcache.writer.queuelength"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancer.java similarity index 77% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancer.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancer.java index fd491fa88e68..3ecd8dc7cfd0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancer.java @@ -57,14 +57,14 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({ LargeTests.class }) -public class TestPrefetchAwareLoadBalancer extends BalancerTestBase { +public class TestCacheAwareLoadBalancer extends BalancerTestBase { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestPrefetchAwareLoadBalancer.class); + HBaseClassTestRule.forClass(TestCacheAwareLoadBalancer.class); - private static final Logger LOG = LoggerFactory.getLogger(TestPrefetchAwareLoadBalancer.class); + private static final Logger LOG = LoggerFactory.getLogger(TestCacheAwareLoadBalancer.class); - private static PrefetchAwareLoadBalancer loadBalancer; + private static CacheAwareLoadBalancer loadBalancer; static List servers; @@ -95,9 +95,9 @@ private static List constructTableDesc(boolean hasBogusTable) { return tds; } - private ServerMetrics mockServerMetricsWithRegionPrefetchInfo(ServerName server, - List regionsOnServer, float currentPrefetchRatio, - List oldPrefechedRegions, int oldPrefetchSize, int regionSize) { + private ServerMetrics mockServerMetricsWithRegionCacheInfo(ServerName server, + List regionsOnServer, float currentCacheRatio, List oldRegionCacheInfo, + int oldRegionCachedSize, int regionSize) { ServerMetrics serverMetrics = mock(ServerMetrics.class); Map regionLoadMap = new TreeMap<>(Bytes.BYTES_COMPARATOR); for (RegionInfo info : regionsOnServer) { @@ -106,16 +106,16 @@ private ServerMetrics mockServerMetricsWithRegionPrefetchInfo(ServerName server, when(rl.getWriteRequestCount()).thenReturn(0L); when(rl.getMemStoreSize()).thenReturn(Size.ZERO); when(rl.getStoreFileSize()).thenReturn(Size.ZERO); - when(rl.getCurrentRegionCachedRatio()).thenReturn(currentPrefetchRatio); + when(rl.getCurrentRegionCachedRatio()).thenReturn(currentCacheRatio); when(rl.getRegionSizeMB()).thenReturn(new Size(regionSize, Size.Unit.MEGABYTE)); regionLoadMap.put(info.getRegionName(), rl); } when(serverMetrics.getRegionMetrics()).thenReturn(regionLoadMap); - Map oldPrefetchInfoMap = new HashMap<>(); - for (RegionInfo info : oldPrefechedRegions) { - oldPrefetchInfoMap.put(info.getEncodedName(), oldPrefetchSize); + Map oldCacheRatioMap = new HashMap<>(); + for (RegionInfo info : oldRegionCacheInfo) { + oldCacheRatioMap.put(info.getEncodedName(), oldRegionCachedSize); } - when(serverMetrics.getRegionCachedInfo()).thenReturn(oldPrefetchInfoMap); + when(serverMetrics.getRegionCachedInfo()).thenReturn(oldCacheRatioMap); return serverMetrics; } @@ -124,15 +124,15 @@ public static void beforeAllTests() throws Exception { servers = generateServers(3); tableDescs = constructTableDesc(false); Configuration conf = HBaseConfiguration.create(); - conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "prefetch_file_list"); - loadBalancer = new PrefetchAwareLoadBalancer(); + conf.set(HConstants.BUCKET_CACHE_PERSISTENT_PATH_KEY, "prefetch_file_list"); + loadBalancer = new CacheAwareLoadBalancer(); loadBalancer.setClusterInfoProvider(new DummyClusterInfoProvider(conf)); loadBalancer.loadConf(conf); } @Test - public void testRegionsNotPrefetchedOnOldServerAndCurrentServer() throws Exception { - // The regions are not prefetched on old server as well as the current server. This causes + public void testRegionsNotCachedOnOldServerAndCurrentServer() throws Exception { + // The regions are not cached on old server as well as the current server. This causes // skewness in the region allocation which should be fixed by the balancer Map> clusterState = new HashMap<>(); @@ -151,11 +151,11 @@ public void testRegionsNotPrefetchedOnOldServerAndCurrentServer() throws Excepti // Mock cluster metrics Map serverMetricsMap = new TreeMap<>(); - serverMetricsMap.put(server0, mockServerMetricsWithRegionPrefetchInfo(server0, regionsOnServer0, + serverMetricsMap.put(server0, mockServerMetricsWithRegionCacheInfo(server0, regionsOnServer0, 0.0f, new ArrayList<>(), 0, 10)); - serverMetricsMap.put(server1, mockServerMetricsWithRegionPrefetchInfo(server1, regionsOnServer1, + serverMetricsMap.put(server1, mockServerMetricsWithRegionCacheInfo(server1, regionsOnServer1, 0.0f, new ArrayList<>(), 0, 10)); - serverMetricsMap.put(server2, mockServerMetricsWithRegionPrefetchInfo(server2, regionsOnServer2, + serverMetricsMap.put(server2, mockServerMetricsWithRegionCacheInfo(server2, regionsOnServer2, 0.0f, new ArrayList<>(), 0, 10)); ClusterMetrics clusterMetrics = mock(ClusterMetrics.class); when(clusterMetrics.getLiveServerMetrics()).thenReturn(serverMetricsMap); @@ -181,9 +181,8 @@ public void testRegionsNotPrefetchedOnOldServerAndCurrentServer() throws Excepti } @Test - public void testRegionsPartiallyPrefetchedOnOldServerAndNotPrefetchedOnCurrentServer() - throws Exception { - // The regions are partially prefetched on old server but not prefetched on the current server + public void testRegionsPartiallyCachedOnOldServerAndNotCachedOnCurrentServer() throws Exception { + // The regions are partially cached on old server but not cached on the current server Map> clusterState = new HashMap<>(); ServerName server0 = servers.get(0); @@ -202,15 +201,14 @@ public void testRegionsPartiallyPrefetchedOnOldServerAndNotPrefetchedOnCurrentSe // Mock cluster metrics // Mock 5 regions from server0 were previously hosted on server1 - List oldPrefetchedRegions = - regionsOnServer0.subList(5, regionsOnServer0.size() - 1); + List oldCachedRegions = regionsOnServer0.subList(5, regionsOnServer0.size() - 1); Map serverMetricsMap = new TreeMap<>(); - serverMetricsMap.put(server0, mockServerMetricsWithRegionPrefetchInfo(server0, regionsOnServer0, + serverMetricsMap.put(server0, mockServerMetricsWithRegionCacheInfo(server0, regionsOnServer0, 0.0f, new ArrayList<>(), 0, 10)); - serverMetricsMap.put(server1, mockServerMetricsWithRegionPrefetchInfo(server1, regionsOnServer1, - 0.0f, oldPrefetchedRegions, 6, 10)); - serverMetricsMap.put(server2, mockServerMetricsWithRegionPrefetchInfo(server2, regionsOnServer2, + serverMetricsMap.put(server1, mockServerMetricsWithRegionCacheInfo(server1, regionsOnServer1, + 0.0f, oldCachedRegions, 6, 10)); + serverMetricsMap.put(server2, mockServerMetricsWithRegionCacheInfo(server2, regionsOnServer2, 0.0f, new ArrayList<>(), 0, 10)); ClusterMetrics clusterMetrics = mock(ClusterMetrics.class); when(clusterMetrics.getLiveServerMetrics()).thenReturn(serverMetricsMap); @@ -233,13 +231,12 @@ public void testRegionsPartiallyPrefetchedOnOldServerAndNotPrefetchedOnCurrentSe // should move 5 regions from server0 to server1 assertEquals(5, regionsMovedFromServer0.size()); assertEquals(5, targetServers.get(server1).size()); - assertTrue(targetServers.get(server1).containsAll(oldPrefetchedRegions)); + assertTrue(targetServers.get(server1).containsAll(oldCachedRegions)); } @Test - public void testRegionsFullyPrefetchedOnOldServerAndNoPrefetchedOnCurrentServers() - throws Exception { - // The regions are fully prefetched on old server + public void testRegionsFullyCachedOnOldServerAndNotCachedOnCurrentServers() throws Exception { + // The regions are fully cached on old server Map> clusterState = new HashMap<>(); ServerName server0 = servers.get(0); @@ -258,15 +255,14 @@ public void testRegionsFullyPrefetchedOnOldServerAndNoPrefetchedOnCurrentServers // Mock cluster metrics // Mock 5 regions from server0 were previously hosted on server1 - List oldPrefetchedRegions = - regionsOnServer0.subList(5, regionsOnServer0.size() - 1); + List oldCachedRegions = regionsOnServer0.subList(5, regionsOnServer0.size() - 1); Map serverMetricsMap = new TreeMap<>(); - serverMetricsMap.put(server0, mockServerMetricsWithRegionPrefetchInfo(server0, regionsOnServer0, + serverMetricsMap.put(server0, mockServerMetricsWithRegionCacheInfo(server0, regionsOnServer0, 0.0f, new ArrayList<>(), 0, 10)); - serverMetricsMap.put(server1, mockServerMetricsWithRegionPrefetchInfo(server1, regionsOnServer1, - 0.0f, oldPrefetchedRegions, 10, 10)); - serverMetricsMap.put(server2, mockServerMetricsWithRegionPrefetchInfo(server2, regionsOnServer2, + serverMetricsMap.put(server1, mockServerMetricsWithRegionCacheInfo(server1, regionsOnServer1, + 0.0f, oldCachedRegions, 10, 10)); + serverMetricsMap.put(server2, mockServerMetricsWithRegionCacheInfo(server2, regionsOnServer2, 0.0f, new ArrayList<>(), 0, 10)); ClusterMetrics clusterMetrics = mock(ClusterMetrics.class); when(clusterMetrics.getLiveServerMetrics()).thenReturn(serverMetricsMap); @@ -289,12 +285,12 @@ public void testRegionsFullyPrefetchedOnOldServerAndNoPrefetchedOnCurrentServers // should move 5 regions from server0 to server1 assertEquals(5, regionsMovedFromServer0.size()); assertEquals(5, targetServers.get(server1).size()); - assertTrue(targetServers.get(server1).containsAll(oldPrefetchedRegions)); + assertTrue(targetServers.get(server1).containsAll(oldCachedRegions)); } @Test - public void testRegionsFullyPrefetchedOnOldAndCurrentServers() throws Exception { - // The regions are fully prefetched on old server + public void testRegionsFullyCachedOnOldAndCurrentServers() throws Exception { + // The regions are fully cached on old server Map> clusterState = new HashMap<>(); ServerName server0 = servers.get(0); @@ -313,15 +309,14 @@ public void testRegionsFullyPrefetchedOnOldAndCurrentServers() throws Exception // Mock cluster metrics // Mock 5 regions from server0 were previously hosted on server1 - List oldPrefetchedRegions = - regionsOnServer0.subList(5, regionsOnServer0.size() - 1); + List oldCachedRegions = regionsOnServer0.subList(5, regionsOnServer0.size() - 1); Map serverMetricsMap = new TreeMap<>(); - serverMetricsMap.put(server0, mockServerMetricsWithRegionPrefetchInfo(server0, regionsOnServer0, + serverMetricsMap.put(server0, mockServerMetricsWithRegionCacheInfo(server0, regionsOnServer0, 1.0f, new ArrayList<>(), 0, 10)); - serverMetricsMap.put(server1, mockServerMetricsWithRegionPrefetchInfo(server1, regionsOnServer1, - 1.0f, oldPrefetchedRegions, 10, 10)); - serverMetricsMap.put(server2, mockServerMetricsWithRegionPrefetchInfo(server2, regionsOnServer2, + serverMetricsMap.put(server1, mockServerMetricsWithRegionCacheInfo(server1, regionsOnServer1, + 1.0f, oldCachedRegions, 10, 10)); + serverMetricsMap.put(server2, mockServerMetricsWithRegionCacheInfo(server2, regionsOnServer2, 1.0f, new ArrayList<>(), 0, 10)); ClusterMetrics clusterMetrics = mock(ClusterMetrics.class); when(clusterMetrics.getLiveServerMetrics()).thenReturn(serverMetricsMap); @@ -344,12 +339,12 @@ public void testRegionsFullyPrefetchedOnOldAndCurrentServers() throws Exception // should move 5 regions from server0 to server1 assertEquals(5, regionsMovedFromServer0.size()); assertEquals(5, targetServers.get(server1).size()); - assertTrue(targetServers.get(server1).containsAll(oldPrefetchedRegions)); + assertTrue(targetServers.get(server1).containsAll(oldCachedRegions)); } @Test - public void testRegionsPartiallyPrefetchedOnOldServerAndCurrentServer() throws Exception { - // The regions are partially prefetched on old server + public void testRegionsPartiallyCachedOnOldServerAndCurrentServer() throws Exception { + // The regions are partially cached on old server Map> clusterState = new HashMap<>(); ServerName server0 = servers.get(0); @@ -368,15 +363,14 @@ public void testRegionsPartiallyPrefetchedOnOldServerAndCurrentServer() throws E // Mock cluster metrics // Mock 5 regions from server0 were previously hosted on server1 - List oldPrefetchedRegions = - regionsOnServer0.subList(5, regionsOnServer0.size() - 1); + List oldCachedRegions = regionsOnServer0.subList(5, regionsOnServer0.size() - 1); Map serverMetricsMap = new TreeMap<>(); - serverMetricsMap.put(server0, mockServerMetricsWithRegionPrefetchInfo(server0, regionsOnServer0, + serverMetricsMap.put(server0, mockServerMetricsWithRegionCacheInfo(server0, regionsOnServer0, 0.2f, new ArrayList<>(), 0, 10)); - serverMetricsMap.put(server1, mockServerMetricsWithRegionPrefetchInfo(server1, regionsOnServer1, - 0.0f, oldPrefetchedRegions, 6, 10)); - serverMetricsMap.put(server2, mockServerMetricsWithRegionPrefetchInfo(server2, regionsOnServer2, + serverMetricsMap.put(server1, mockServerMetricsWithRegionCacheInfo(server1, regionsOnServer1, + 0.0f, oldCachedRegions, 6, 10)); + serverMetricsMap.put(server2, mockServerMetricsWithRegionCacheInfo(server2, regionsOnServer2, 1.0f, new ArrayList<>(), 0, 10)); ClusterMetrics clusterMetrics = mock(ClusterMetrics.class); when(clusterMetrics.getLiveServerMetrics()).thenReturn(serverMetricsMap); @@ -398,6 +392,6 @@ public void testRegionsPartiallyPrefetchedOnOldServerAndCurrentServer() throws E } assertEquals(5, regionsMovedFromServer0.size()); assertEquals(5, targetServers.get(server1).size()); - assertTrue(targetServers.get(server1).containsAll(oldPrefetchedRegions)); + assertTrue(targetServers.get(server1).containsAll(oldCachedRegions)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancerCostFunctions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancerCostFunctions.java new file mode 100644 index 000000000000..448e576b1bc7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestCacheAwareLoadBalancerCostFunctions.java @@ -0,0 +1,316 @@ +/* + * 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.assertTrue; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Pair; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MasterTests.class, MediumTests.class }) +public class TestCacheAwareLoadBalancerCostFunctions extends StochasticBalancerTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCacheAwareLoadBalancerCostFunctions.class); + + // Mapping of test -> expected cache cost + private final float[] expectedCacheCost = { 0.0f, 0.0f, 0.5f, 1.0f, 0.0f, 0.572f, 0.0f, 0.075f }; + + /** + * Data set to testCacheCost: [test][0][0] = mapping of server to number of regions it hosts + * [test][region + 1][0] = server that region is hosted on [test][region + 1][server + 1] = size + * of region cached on server + */ + private final int[][][] clusterRegionCacheRatioMocks = new int[][][] { + // Test 1: each region is entirely on server that hosts it + // Cost of moving the regions in this case should be high as the regions are fully cached + // on the server they are currently hosted on + new int[][] { new int[] { 2, 1, 1 }, // Server 0 has 2, server 1 has 1 and server 2 has 1 + // region(s) hosted respectively + new int[] { 0, 100, 0, 0 }, // region 0 is hosted and cached only on server 0 + new int[] { 0, 100, 0, 0 }, // region 1 is hosted and cached only on server 0 + new int[] { 1, 0, 100, 0 }, // region 2 is hosted and cached only on server 1 + new int[] { 2, 0, 0, 100 }, // region 3 is hosted and cached only on server 2 + }, + + // Test 2: each region is cached completely on the server it is currently hosted on, + // but it was also cached on some other server historically + // Cost of moving the regions in this case should be high as the regions are fully cached + // on the server they are currently hosted on. Although, the regions were previously hosted and + // cached on some other server, since they are completely cached on the new server, + // there is no need to move the regions back to the previously hosting cluster + new int[][] { new int[] { 1, 2, 1 }, // Server 0 has 1, server 1 has 2 and server 2 has 1 + // region(s) hosted respectively + new int[] { 0, 100, 0, 100 }, // region 0 is hosted and currently cached on server 0, + // but previously cached completely on server 2 + new int[] { 1, 100, 100, 0 }, // region 1 is hosted and currently cached on server 1, + // but previously cached completely on server 0 + new int[] { 1, 0, 100, 100 }, // region 2 is hosted and currently cached on server 1, + // but previously cached on server 2 + new int[] { 2, 0, 100, 100 }, // region 3 is hosted and currently cached on server 2, + // but previously cached on server 1 + }, + + // Test 3: The regions were hosted and fully cached on a server but later moved to other + // because of server crash procedure. The regions are partially cached on the server they + // are currently hosted on + new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 50, 0, 100 }, // Region 0 is currently + // hosted and partially + // cached on + // server 0, but was fully + // cached on server 2 + // previously + new int[] { 1, 100, 50, 0 }, // Region 1 is currently hosted and partially cached on + // server 1, but was fully cached on server 0 previously + new int[] { 1, 0, 50, 100 }, // Region 2 is currently hosted and partially cached on + // server 1, but was fully cached on server 2 previously + new int[] { 2, 0, 100, 50 }, // Region 3 is currently hosted and partially cached on + // server 2, but was fully cached on server 1 previously + }, + + // Test 4: The regions were hosted and fully cached on a server, but later moved to other + // server because of server crash procedure. The regions are not at all cached on the server + // they are currently hosted on + new int[][] { new int[] { 1, 1, 2 }, new int[] { 0, 0, 0, 100 }, // Region 0 is currently hosted + // but not cached on server + // 0, + // but was fully cached on + // server 2 previously + new int[] { 1, 100, 0, 0 }, // Region 1 is currently hosted but not cached on server 1, + // but was fully cached on server 0 previously + new int[] { 2, 0, 100, 0 }, // Region 2 is currently hosted but not cached on server 2, + // but was fully cached on server 1 previously + new int[] { 2, 100, 0, 0 }, // Region 3 is currently hosted but not cached on server 2, + // but was fully cached on server 1 previously + }, + + // Test 5: The regions were partially cached on old servers, before moving to the new server + // where also, they are partially cached + new int[][] { new int[] { 2, 1, 1 }, new int[] { 0, 50, 50, 0 }, // Region 0 is hosted and + // partially cached on + // server 0, but + // was previously hosted and + // partially cached on + // server 1 + new int[] { 0, 50, 0, 50 }, // Region 1 is hosted and partially cached on server 0, but + // was previously hosted and partially cached on server 2 + new int[] { 1, 0, 50, 50 }, // Region 2 is hosted and partially cached on server 1, but + // was previously hosted and partially cached on server 2 + new int[] { 2, 0, 50, 50 }, // Region 3 is hosted and partially cached on server 2, but + // was previously hosted and partially cached on server 1 + }, + + // Test 6: The regions are less cached on the new servers as compared to what they were + // cached on the server before they were moved to the new servers + new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 30, 70, 0 }, // Region 0 is hosted and + // cached 30% on server 0, + // but was + // previously hosted and + // cached 70% on server 1 + new int[] { 1, 70, 30, 0 }, // Region 1 is hosted and cached 30% on server 1, but was + // previously hosted and cached 70% on server 0 + new int[] { 1, 0, 30, 70 }, // Region 2 is hosted and cached 30% on server 1, but was + // previously hosted and cached 70% on server 2 + new int[] { 2, 0, 70, 30 }, // Region 3 is hosted and cached 30% on server 2, but was + // previously hosted and cached 70% on server 1 + }, + + // Test 7: The regions are more cached on the new servers as compared to what they were + // cached on the server before they were moved to the new servers + new int[][] { new int[] { 2, 1, 1 }, new int[] { 0, 80, 20, 0 }, // Region 0 is hosted and 80% + // cached on server 0, but + // was + // previously hosted and 20% + // cached on server 1 + new int[] { 0, 80, 0, 20 }, // Region 1 is hosted and 80% cached on server 0, but was + // previously hosted and 20% cached on server 2 + new int[] { 1, 20, 80, 0 }, // Region 2 is hosted and 80% cached on server 1, but was + // previously hosted and 20% cached on server 0 + new int[] { 2, 0, 20, 80 }, // Region 3 is hosted and 80% cached on server 2, but was + // previously hosted and 20% cached on server 1 + }, + + // Test 8: The regions are randomly assigned to the server with some regions historically + // hosted on other region servers + new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 34, 0, 58 }, // Region 0 is hosted and + // partially cached on + // server 0, + // but was previously hosted + // and partially cached on + // server 2 + // current cache ratio < + // historical cache ratio + new int[] { 1, 78, 100, 0 }, // Region 1 is hosted and fully cached on server 1, + // but was previously hosted and partially cached on server 0 + // current cache ratio > historical cache ratio + new int[] { 1, 66, 66, 0 }, // Region 2 is hosted and partially cached on server 1, + // but was previously hosted and partially cached on server 0 + // current cache ratio == historical cache ratio + new int[] { 2, 0, 0, 96 }, // Region 3 is hosted and partially cached on server 0 + // No historical cache ratio + }, }; + + private static Configuration storedConfiguration; + + private CacheAwareLoadBalancer loadBalancer = new CacheAwareLoadBalancer(); + + @BeforeClass + public static void saveInitialConfiguration() { + storedConfiguration = new Configuration(conf); + } + + @Before + public void beforeEachTest() { + conf = new Configuration(storedConfiguration); + loadBalancer.loadConf(conf); + } + + @Test + public void testVerifyCacheAwareSkewnessCostFunctionEnabled() { + CacheAwareLoadBalancer lb = new CacheAwareLoadBalancer(); + lb.loadConf(conf); + assertTrue(Arrays.asList(lb.getCostFunctionNames()) + .contains(CacheAwareLoadBalancer.CacheAwareRegionSkewnessCostFunction.class.getSimpleName())); + } + + @Test + public void testVerifyCacheAwareSkewnessCostFunctionDisabled() { + conf.setFloat( + CacheAwareLoadBalancer.CacheAwareRegionSkewnessCostFunction.REGION_COUNT_SKEW_COST_KEY, 0.0f); + + CacheAwareLoadBalancer lb = new CacheAwareLoadBalancer(); + lb.loadConf(conf); + + assertFalse(Arrays.asList(lb.getCostFunctionNames()) + .contains(CacheAwareLoadBalancer.CacheAwareRegionSkewnessCostFunction.class.getSimpleName())); + } + + @Test + public void testVerifyCacheCostFunctionEnabled() { + conf.set(HConstants.BUCKET_CACHE_PERSISTENT_PATH_KEY, "/tmp/prefetch.persistence"); + + CacheAwareLoadBalancer lb = new CacheAwareLoadBalancer(); + lb.loadConf(conf); + + assertTrue(Arrays.asList(lb.getCostFunctionNames()) + .contains(CacheAwareLoadBalancer.CacheAwareCostFunction.class.getSimpleName())); + } + + @Test + public void testVerifyCacheCostFunctionDisabledByNoBucketCachePersistence() { + assertFalse(Arrays.asList(loadBalancer.getCostFunctionNames()) + .contains(CacheAwareLoadBalancer.CacheAwareCostFunction.class.getSimpleName())); + } + + @Test + public void testVerifyCacheCostFunctionDisabledByNoMultiplier() { + conf.set(HConstants.BUCKET_CACHE_PERSISTENT_PATH_KEY, "/tmp/prefetch.persistence"); + conf.setFloat("hbase.master.balancer.stochastic.cacheCost", 0.0f); + assertFalse(Arrays.asList(loadBalancer.getCostFunctionNames()) + .contains(CacheAwareLoadBalancer.CacheAwareCostFunction.class.getSimpleName())); + } + + @Test + public void testCacheCost() { + conf.set(HConstants.BUCKET_CACHE_PERSISTENT_PATH_KEY, "/tmp/prefetch.persistence"); + CacheAwareLoadBalancer.CacheAwareCostFunction costFunction = + new CacheAwareLoadBalancer.CacheAwareCostFunction(conf); + + for (int test = 0; test < clusterRegionCacheRatioMocks.length; test++) { + int[][] clusterRegionLocations = clusterRegionCacheRatioMocks[test]; + MockClusterForCacheCost cluster = new MockClusterForCacheCost(clusterRegionLocations); + costFunction.prepare(cluster); + double cost = costFunction.cost(); + assertEquals(expectedCacheCost[test], cost, 0.01); + } + } + + private class MockClusterForCacheCost extends BalancerClusterState { + private final Map, Float> regionServerCacheRatio = new HashMap<>(); + + public MockClusterForCacheCost(int[][] regionsArray) { + // regions[0] is an array where index = serverIndex and value = number of regions + super(mockClusterServersUnsorted(regionsArray[0], 1), null, null, null, null); + Map> oldCacheRatio = new HashMap<>(); + for (int i = 1; i < regionsArray.length; i++) { + int regionIndex = i - 1; + for (int j = 1; j < regionsArray[i].length; j++) { + int serverIndex = j - 1; + float cacheRatio = (float) regionsArray[i][j] / 100; + regionServerCacheRatio.put(new Pair<>(regionIndex, serverIndex), cacheRatio); + if (cacheRatio > 0.0f && serverIndex != regionsArray[i][0]) { + // This is the historical cacheRatio value + oldCacheRatio.put(regions[regionIndex].getEncodedName(), + new Pair<>(servers[serverIndex], cacheRatio)); + } + } + } + regionCacheRatioOnOldServerMap = oldCacheRatio; + } + + @Override + public int getTotalRegionHFileSizeMB(int region) { + return 1; + } + + @Override + protected float getRegionCacheRatioOnRegionServer(int region, int regionServerIndex) { + float cacheRatio = 0.0f; + + // Get the cache ratio if the region is currently hosted on this server + if (regionServerIndex == regionIndexToServerIndex[region]) { + return regionServerCacheRatio.get(new Pair<>(region, regionServerIndex)); + } + + // Region is not currently hosted on this server. Check if the region was cached on this + // server earlier. This can happen when the server was shutdown and the cache was persisted. + // Search using the index name and server name and not the index id and server id as these + // ids may change when a server is marked as dead or a new server is added. + String regionEncodedName = regions[region].getEncodedName(); + ServerName serverName = servers[regionServerIndex]; + if ( + regionCacheRatioOnOldServerMap != null + && regionCacheRatioOnOldServerMap.containsKey(regionEncodedName) + ) { + Pair serverCacheRatio = + regionCacheRatioOnOldServerMap.get(regionEncodedName); + if (ServerName.isSameAddress(serverName, serverCacheRatio.getFirst())) { + cacheRatio = serverCacheRatio.getSecond(); + regionCacheRatioOnOldServerMap.remove(regionEncodedName); + } + } + return cacheRatio; + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancerCostFunctions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancerCostFunctions.java deleted file mode 100644 index 9568d78c1577..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchAwareLoadBalancerCostFunctions.java +++ /dev/null @@ -1,319 +0,0 @@ -/* - * 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.assertTrue; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.util.Pair; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category({ MasterTests.class, MediumTests.class }) -public class TestPrefetchAwareLoadBalancerCostFunctions extends StochasticBalancerTestBase { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestPrefetchAwareLoadBalancerCostFunctions.class); - - // Mapping of prefetch test -> expected prefetch - private final float[] expectedPrefetch = { 0.0f, 0.0f, 0.5f, 1.0f, 0.0f, 0.572f, 0.0f, 0.075f }; - - /** - * Data set to testPrefetchCost: [test][0][0] = mapping of server to number of regions it hosts - * [test][region + 1][0] = server that region is hosted on [test][region + 1][server + 1] = - * prefetch of that region on server - */ - private final int[][][] clusterRegionPrefetchMocks = new int[][][] { - // Test 1: each region is entirely on server that hosts it - // Cost of moving the regions in this case should be high as the regions are fully prefetched - // on the server they are currently hosted on - new int[][] { new int[] { 2, 1, 1 }, // Server 0 has 2, server 1 has 1 and server 2 has 1 - // region(s) hosted respectively - new int[] { 0, 100, 0, 0 }, // region 0 is hosted and prefetched only on server 0 - new int[] { 0, 100, 0, 0 }, // region 1 is hosted and prefetched only on server 0 - new int[] { 1, 0, 100, 0 }, // region 2 is hosted and prefetched only on server 1 - new int[] { 2, 0, 0, 100 }, // region 3 is hosted and prefetched only on server 2 - }, - - // Test 2: each region is prefetched completely on the server it is currently hosted on, - // but it was also prefetched on some other server historically - // Cost of moving the regions in this case should be high as the regions are fully prefetched - // on the server they are currently hosted on. Although, the regions were previously hosted and - // prefetched on some other server, since they are completely prefetched on the new server, - // there is no need to move the regions back to the previously hosting cluster - new int[][] { new int[] { 1, 2, 1 }, // Server 0 has 1, server 1 has 2 and server 2 has 1 - // region(s) hosted respectively - new int[] { 0, 100, 0, 100 }, // region 0 is hosted and currently prefetched on server 0, - // but previously prefetched completely on server 2 - new int[] { 1, 100, 100, 0 }, // region 1 is hosted and currently prefetched on server 1, - // but previously prefetched completely on server 0 - new int[] { 1, 0, 100, 100 }, // region 2 is hosted and currently prefetched on server 1, - // but previously prefetched on server 2 - new int[] { 2, 0, 100, 100 }, // region 3 is hosted and currently prefetched on server 2, - // but previously prefetched on server 1 - }, - - // Test 3: The regions were hosted and fully prefetched on a server but later moved to other - // because of server crash procedure. The regions are partially prefetched on the server they - // are currently hosted on - new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 50, 0, 100 }, // Region 0 is currently - // hosted and partially - // prefetched on - // server 0, but was fully - // prefetched on server 2 - // previously - new int[] { 1, 100, 50, 0 }, // Region 1 is currently hosted and partially prefetched on - // server 1, but was fully prefetched on server 0 previously - new int[] { 1, 0, 50, 100 }, // Region 2 is currently hosted and partially prefetched on - // server 1, but was fully prefetched on server 2 previously - new int[] { 2, 0, 100, 50 }, // Region 3 is currently hosted and partially prefetched on - // server 2, but was fully prefetched on server 1 previously - }, - - // Test 4: The regions were hosted and fully prefetched on a server, but later moved to other - // server because of server crash procedure. The regions are not at all prefetched on the server - // they are currently hosted on - new int[][] { new int[] { 1, 1, 2 }, new int[] { 0, 0, 0, 100 }, // Region 0 is currently hosted - // but not prefetched on server - // 0, - // but was fully prefetched on - // server 2 previously - new int[] { 1, 100, 0, 0 }, // Region 1 is currently hosted but not prefetched on server 1, - // but was fully prefetched on server 0 previously - new int[] { 2, 0, 100, 0 }, // Region 2 is currently hosted but not prefetched on server 2, - // but was fully prefetched on server 1 previously - new int[] { 2, 100, 0, 0 }, // Region 3 is currently hosted but not prefetched on server 2, - // but was fully prefetched on server 1 previously - }, - - // Test 5: The regions were partially prefetched on old servers, before moving to the new server - // where also, they are partially prefetched - new int[][] { new int[] { 2, 1, 1 }, new int[] { 0, 50, 50, 0 }, // Region 0 is hosted and - // partially prefetched on - // server 0, but - // was previously hosted and - // partially prefetched on - // server 1 - new int[] { 0, 50, 0, 50 }, // Region 1 is hosted and partially prefetched on server 0, but - // was previously hosted and partially prefetched on server 2 - new int[] { 1, 0, 50, 50 }, // Region 2 is hosted and partially prefetched on server 1, but - // was previously hosted and partially prefetched on server 2 - new int[] { 2, 0, 50, 50 }, // Region 3 is hosted and partially prefetched on server 2, but - // was previously hosted and partially prefetched on server 1 - }, - - // Test 6: The regions are less prefetched on the new servers as compared to what they were - // prefetched on the server before they were moved to the new servers - new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 30, 70, 0 }, // Region 0 is hosted and - // prefetched 30% on server 0, - // but was - // previously hosted and - // prefetched 70% on server 1 - new int[] { 1, 70, 30, 0 }, // Region 1 is hosted and prefetched 30% on server 1, but was - // previously hosted and prefetched 70% on server 0 - new int[] { 1, 0, 30, 70 }, // Region 2 is hosted and prefetched 30% on server 1, but was - // previously hosted and prefetched 70% on server 2 - new int[] { 2, 0, 70, 30 }, // Region 3 is hosted and prefetched 30% on server 2, but was - // previously hosted and prefetched 70% on server 1 - }, - - // Test 7: The regions are more prefetched on the new servers as compared to what they were - // prefetched on the server before they were moved to the new servers - new int[][] { new int[] { 2, 1, 1 }, new int[] { 0, 80, 20, 0 }, // Region 0 is hosted and 80% - // prefetched on server 0, but - // was - // previously hosted and 20% - // prefetched on server 1 - new int[] { 0, 80, 0, 20 }, // Region 1 is hosted and 80% prefetched on server 0, but was - // previously hosted and 20% prefetched on server 2 - new int[] { 1, 20, 80, 0 }, // Region 2 is hosted and 80% prefetched on server 1, but was - // previously hosted and 20% prefetched on server 0 - new int[] { 2, 0, 20, 80 }, // Region 3 is hosted and 80% prefetched on server 2, but was - // previously hosted and 20% prefetched on server 1 - }, - - // Test 8: The regions are randomly assigned to the server with some regions historically - // hosted on other region servers - new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 34, 0, 58 }, // Region 0 is hosted and - // partially prefetched on - // server 0, - // but was previously hosted - // and partially prefetched on - // server 2 - // current prefetch < - // historical prefetch - new int[] { 1, 78, 100, 0 }, // Region 1 is hosted and fully prefetched on server 1, - // but was previously hosted and partially prefetched on server 0 - // current prefetch > historical prefetch - new int[] { 1, 66, 66, 0 }, // Region 2 is hosted and partially prefetched on server 1, - // but was previously hosted and partially prefetched on server 0 - // current prefetch == historical prefetch - new int[] { 2, 0, 0, 96 }, // Region 3 is hosted and partially prefetched on server 0 - // No historical prefetch - }, }; - - private static Configuration storedConfiguration; - - private PrefetchAwareLoadBalancer loadBalancer = new PrefetchAwareLoadBalancer(); - - @BeforeClass - public static void saveInitialConfiguration() { - storedConfiguration = new Configuration(conf); - } - - @Before - public void beforeEachTest() { - conf = new Configuration(storedConfiguration); - loadBalancer.loadConf(conf); - } - - @Test - public void testVerifyPrefetchAwareSkewnessCostFunctionEnabled() { - PrefetchAwareLoadBalancer lb = new PrefetchAwareLoadBalancer(); - lb.loadConf(conf); - assertTrue(Arrays.asList(lb.getCostFunctionNames()).contains( - PrefetchAwareLoadBalancer.PrefetchAwareRegionSkewnessCostFunction.class.getSimpleName())); - } - - @Test - public void testVerifyPrefetchAwareSkewnessCostFunctionDisabled() { - conf.setFloat( - PrefetchAwareLoadBalancer.PrefetchAwareRegionSkewnessCostFunction.REGION_COUNT_SKEW_COST_KEY, - 0.0f); - - PrefetchAwareLoadBalancer lb = new PrefetchAwareLoadBalancer(); - lb.loadConf(conf); - - assertFalse(Arrays.asList(lb.getCostFunctionNames()).contains( - PrefetchAwareLoadBalancer.PrefetchAwareRegionSkewnessCostFunction.class.getSimpleName())); - } - - @Test - public void testVerifyPrefetchCostFunctionEnabled() { - conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch.persistence"); - - PrefetchAwareLoadBalancer lb = new PrefetchAwareLoadBalancer(); - lb.loadConf(conf); - - assertTrue(Arrays.asList(lb.getCostFunctionNames()) - .contains(PrefetchAwareLoadBalancer.PrefetchCacheCostFunction.class.getSimpleName())); - } - - @Test - public void testVerifyPrefetchCostFunctionDisabledByNoPersistencePathKey() { - assertFalse(Arrays.asList(loadBalancer.getCostFunctionNames()) - .contains(PrefetchAwareLoadBalancer.PrefetchCacheCostFunction.class.getSimpleName())); - } - - @Test - public void testVerifyPrefetchCostFunctionDisabledByNoMultiplier() { - conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch.persistence"); - conf.setFloat("hbase.master.balancer.stochastic.prefetchCacheCost", 0.0f); - assertFalse(Arrays.asList(loadBalancer.getCostFunctionNames()) - .contains(PrefetchAwareLoadBalancer.PrefetchCacheCostFunction.class.getSimpleName())); - } - - @Test - public void testPrefetchCost() { - conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch.persistence"); - PrefetchAwareLoadBalancer.PrefetchCacheCostFunction costFunction = - new PrefetchAwareLoadBalancer.PrefetchCacheCostFunction(conf); - - for (int test = 0; test < clusterRegionPrefetchMocks.length; test++) { - int[][] clusterRegionLocations = clusterRegionPrefetchMocks[test]; - TestPrefetchAwareLoadBalancerCostFunctions.MockClusterForPrefetch cluster = - new TestPrefetchAwareLoadBalancerCostFunctions.MockClusterForPrefetch( - clusterRegionLocations); - costFunction.prepare(cluster); - double cost = costFunction.cost(); - assertEquals(expectedPrefetch[test], cost, 0.01); - } - } - - private class MockClusterForPrefetch extends BalancerClusterState { - private final Map, Float> regionServerPrefetch = new HashMap<>(); - - public MockClusterForPrefetch(int[][] regionsArray) { - // regions[0] is an array where index = serverIndex and value = number of regions - super(mockClusterServersUnsorted(regionsArray[0], 1), null, null, null, null); - Map> oldPrefetchRatio = new HashMap<>(); - for (int i = 1; i < regionsArray.length; i++) { - int regionIndex = i - 1; - for (int j = 1; j < regionsArray[i].length; j++) { - int serverIndex = j - 1; - float prefetch = (float) regionsArray[i][j] / 100; - regionServerPrefetch.put(new Pair<>(regionIndex, serverIndex), prefetch); - if (prefetch > 0.0f && serverIndex != regionsArray[i][0]) { - // This is the historical prefetch value - oldPrefetchRatio.put(regions[regionIndex].getEncodedName(), - new Pair<>(servers[serverIndex], prefetch)); - } - } - } - oldRegionServerPrefetchRatio = oldPrefetchRatio; - } - - @Override - public int getTotalRegionHFileSizeMB(int region) { - return 1; - } - - @Override - protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) { - float prefetchRatio = 0.0f; - - // Get the prefetch cache ratio if the region is currently hosted on this server - if (regionServerIndex == regionIndexToServerIndex[region]) { - return regionServerPrefetch.get(new Pair<>(region, regionServerIndex)); - } - - // Region is not currently hosted on this server. Check if the region was prefetched on this - // server earlier. This can happen when the server was shutdown and the cache was persisted. - // Search using the index name and server name and not the index id and server id as these - // ids may change when a server is marked as dead or a new server is added. - String regionEncodedName = regions[region].getEncodedName(); - ServerName serverName = servers[regionServerIndex]; - if ( - oldRegionServerPrefetchRatio != null - && oldRegionServerPrefetchRatio.containsKey(regionEncodedName) - ) { - Pair serverPrefetchRatio = - oldRegionServerPrefetchRatio.get(regionEncodedName); - if (ServerName.isSameAddress(serverName, serverPrefetchRatio.getFirst())) { - prefetchRatio = serverPrefetchRatio.getSecond(); - oldRegionServerPrefetchRatio.remove(regionEncodedName); - } - } - return prefetchRatio; - } - } -}