From aa22468dfc63c976c1641cfe839ded9b18d1acec Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Fri, 23 Sep 2022 21:50:43 +0530 Subject: [PATCH 01/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions --- .../master/balancer/BalancerClusterState.java | 60 +++++++++ .../master/balancer/BalancerRegionLoad.java | 4 + .../PrefetchBasedCandidateGenerator.java | 79 ++++++++++++ .../balancer/PrefetchCacheCostFunction.java | 82 ++++++++++++ .../balancer/StochasticLoadBalancer.java | 13 +- .../balancer/TestStochasticLoadBalancer.java | 122 ++++++++++++++++++ .../apache/hadoop/hbase/RegionMetrics.java | 3 + .../hadoop/hbase/RegionMetricsBuilder.java | 20 ++- .../main/protobuf/server/ClusterStatus.proto | 3 + .../hbase/regionserver/HRegionServer.java | 14 +- .../master/TestRegionsRecoveryChore.java | 3 + 11 files changed, 398 insertions(+), 5 deletions(-) create mode 100644 hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java create mode 100644 hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.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..dcc4ea96895e 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,6 +115,11 @@ class BalancerClusterState { // Maps localityType -> region -> [server|rack]Index with highest locality private int[][] regionsToMostLocalEntities; + // Maps region -> serverIndex -> prefetch ratio of a region on a server + private float[][] regionServerPrefetchRatio; + // Maps region -> serverIndex with best prefect ratio + private int[] regionServerWithBestPrefetchRatio; + static class DefaultRackManager extends RackManager { @Override public String getRack(ServerName server) { @@ -553,6 +558,61 @@ enum LocalityType { RACK } + public float getOrComputeWeightedPrefetchRatio(int region, int server) { + return getRegionSizeMB(region) * getOrComputeRegionPrefetchRatio()[region][server]; + } + + private float[][] getOrComputeRegionPrefetchRatio() { + if (regionServerWithBestPrefetchRatio == null || regionServerPrefetchRatio == null) { + computeRegionServerPrefetchRatio(); + } + return regionServerPrefetchRatio; + } + + public int[] getOrComputeServerWithBestPrefetchRatio() { + if (regionServerWithBestPrefetchRatio == null || regionServerPrefetchRatio == null) { + computeRegionServerPrefetchRatio(); + } + return regionServerWithBestPrefetchRatio; + } + + private void computeRegionServerPrefetchRatio() { + regionServerPrefetchRatio = new float[numRegions][numServers]; + regionServerWithBestPrefetchRatio = 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); + regionServerPrefetchRatio[region][server] = prefetchRatio; + if (prefetchRatio > bestPrefetchRatio) { + serverWithBestPrefetchRatio = server; + bestPrefetchRatio = prefetchRatio; + } + } + regionServerWithBestPrefetchRatio[region] = serverWithBestPrefetchRatio; + } + } + + protected float getRegionServerPrefetchRatio(int region, int server) { + float prefetchRatio = 0.0f; + + // For every region on this region server get the region load + for (int regionIndex : regionsPerServer[server]) { + if (region != regionIndex) { + continue; + } + Deque regionLoadList = regionLoads[regionIndex]; + + // Found the region load. Find the prefetch ratio for this region load + prefetchRatio = regionLoadList.isEmpty() ? 0.0f : + regionLoadList.getLast().getPrefetchCacheRatio(); + break; + } + return prefetchRatio; + } + public void doAction(BalanceAction action) { switch (action.getType()) { case NULL: 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..25dff972f526 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,7 @@ class BalancerRegionLoad { private final long writeRequestsCount; private final int memStoreSizeMB; private final int storefileSizeMB; + private final float prefetchCacheRatio; BalancerRegionLoad(RegionMetrics regionMetrics) { readRequestsCount = regionMetrics.getReadRequestCount(); @@ -41,6 +42,7 @@ class BalancerRegionLoad { writeRequestsCount = regionMetrics.getWriteRequestCount(); memStoreSizeMB = (int) regionMetrics.getMemStoreSize().get(Size.Unit.MEGABYTE); storefileSizeMB = (int) regionMetrics.getStoreFileSize().get(Size.Unit.MEGABYTE); + prefetchCacheRatio = regionMetrics.getPrefetchCacheRatio(); } public long getReadRequestsCount() { @@ -62,4 +64,6 @@ public int getMemStoreSizeMB() { public int getStorefileSizeMB() { return storefileSizeMB; } + + public float getPrefetchCacheRatio() { return prefetchCacheRatio; } } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java new file mode 100644 index 000000000000..ee5125a57675 --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.master.balancer; + +import java.util.Optional; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +class PrefetchBasedCandidateGenerator extends CandidateGenerator{ + @Override + BalanceAction generate(BalancerClusterState cluster) { + // iterate through regions until you find one that is not on ideal host + // start from a random point to avoid always balance the regions in front + if (cluster.numRegions > 0) { + int startIndex = ThreadLocalRandom.current().nextInt(cluster.numRegions); + for (int i = 0; i < cluster.numRegions; i++) { + int region = (startIndex + i) % cluster.numRegions; + int currentServer = cluster.regionIndexToServerIndex[region]; + if ( + currentServer != cluster.getOrComputeServerWithBestPrefetchRatio()[region] + ) { + Optional potential = tryMoveOrSwap(cluster, + currentServer, region, cluster.getOrComputeServerWithBestPrefetchRatio()[region]); + if (potential.isPresent()) { + return potential.get(); + } + } + } + } + return BalanceAction.NULL_ACTION; + } + + private Optional tryMoveOrSwap(BalancerClusterState cluster, + int fromServer, int fromRegion, int toServer) { + // Try move first. We know apriori fromRegion has the highest locality on toServer + if (cluster.serverHasTooFewRegions(toServer)) { + return Optional.of(getAction(fromServer, fromRegion, toServer, -1)); + } + // Compare prefetch gain/loss from swapping fromRegion with regions on toServer + float fromRegionPrefetchDelta = getWeightedPrefetch(cluster, fromRegion, toServer) + - getWeightedPrefetch(cluster, fromRegion, fromServer); + int toServertotalRegions = cluster.regionsPerServer[toServer].length; + if (toServertotalRegions > 0) { + int startIndex = ThreadLocalRandom.current().nextInt(toServertotalRegions); + for (int i = 0; i < toServertotalRegions; i++) { + int toRegionIndex = (startIndex + i) % toServertotalRegions; + int toRegion = cluster.regionsPerServer[toServer][toRegionIndex]; + float toRegionPrefetchDelta = getWeightedPrefetch(cluster, toRegion, fromServer) + - getWeightedPrefetch(cluster, toRegion, toServer); + // If prefetch would remain neutral or improve, attempt the swap + if (fromRegionPrefetchDelta + toRegionPrefetchDelta >= 0) { + return Optional.of(getAction(fromServer, fromRegion, toServer, toRegion)); + } + } + } + return Optional.empty(); + } + + private float getWeightedPrefetch(BalancerClusterState cluster, int region, int server) { + return cluster.getOrComputeWeightedPrefetchRatio(region, server); + } +} diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java new file mode 100644 index 000000000000..3738f48f44ff --- /dev/null +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.master.balancer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Compute the cost of a potential cluster configuration based on + * the number of HFile's already cached in the bucket cache + */ +@InterfaceAudience.Private +public class PrefetchCacheCostFunction extends CostFunction { + private String prefetchedFileListPath; + private float prefetchRatio; + private float bestPrefetchRatio; + + public static final String PREFETCH_PERSISTENCE_PATH_KEY = "hbase.prefetch.file-list.path"; + + PrefetchCacheCostFunction(Configuration conf) { + prefetchedFileListPath = conf.get(PREFETCH_PERSISTENCE_PATH_KEY); + this.setMultiplier(prefetchedFileListPath == null ? 0 : 1); + prefetchRatio = 0.0f; + bestPrefetchRatio = 0.0f; + } + + @Override + void prepare(BalancerClusterState cluster) { + super.prepare(cluster); + prefetchRatio = 0.0f; + bestPrefetchRatio = 0.0f; + + for (int region = 0; region < cluster.numRegions; region++) { + prefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region, + cluster.regionIndexToServerIndex[region]); + bestPrefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region, + cluster.getOrComputeServerWithBestPrefetchRatio()[region]); + } + prefetchRatio = bestPrefetchRatio == 0.0f ? 1.0f : + prefetchRatio/bestPrefetchRatio; + } + + @Override + protected double cost() { + return 1 - prefetchRatio; + } + + @Override + protected void regionMoved(int region, int oldServer, int newServer) { + float oldServerPrefetch = getWeightedPrefetchRatio(region, oldServer); + float newServerPrefetch = getWeightedPrefetchRatio(region, newServer); + float prefetchDelta = newServerPrefetch - oldServerPrefetch; + float normalizeDelta = bestPrefetchRatio == 0.0f ? 0.0f : + prefetchDelta/bestPrefetchRatio; + prefetchRatio += normalizeDelta; + } + + @Override + public final void updateWeight(double[] weights) { + weights[StochasticLoadBalancer.GeneratorType.PREFETCH.ordinal()] += cost(); + } + + private float getWeightedPrefetchRatio(int region, int server) { + return cluster.getOrComputeWeightedPrefetchRatio(region, server); + } +} 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..f8f970d9cced 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 @@ -153,6 +153,9 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { private RackLocalityCostFunction rackLocalityCost; private RegionReplicaHostCostFunction regionReplicaHostCostFunction; private RegionReplicaRackCostFunction regionReplicaRackCostFunction; + private PrefetchCacheCostFunction prefetchCacheCost; + private PrefetchBasedCandidateGenerator prefetchCandidateGenerator; + protected List candidateGenerators; @@ -160,7 +163,8 @@ public enum GeneratorType { RANDOM, LOAD, LOCALITY, - RACK + RACK, + PREFETCH } /** @@ -221,6 +225,7 @@ protected List createCandidateGenerators() { candidateGenerators.add(GeneratorType.LOCALITY.ordinal(), localityCandidateGenerator); candidateGenerators.add(GeneratorType.RACK.ordinal(), new RegionReplicaRackCandidateGenerator()); + candidateGenerators.add(GeneratorType.PREFETCH.ordinal(), prefetchCandidateGenerator); return candidateGenerators; } @@ -237,6 +242,8 @@ protected void loadConf(Configuration conf) { localityCandidateGenerator = new LocalityBasedCandidateGenerator(); localityCost = new ServerLocalityCostFunction(conf); rackLocalityCost = new RackLocalityCostFunction(conf); + prefetchCacheCost = new PrefetchCacheCostFunction(conf); + prefetchCandidateGenerator = new PrefetchBasedCandidateGenerator(); this.candidateGenerators = createCandidateGenerators(); @@ -256,6 +263,7 @@ protected void loadConf(Configuration conf) { addCostFunction(new WriteRequestCostFunction(conf)); addCostFunction(new MemStoreSizeCostFunction(conf)); addCostFunction(new StoreFileCostFunction(conf)); + addCostFunction(prefetchCacheCost); loadCustomCostFunctions(conf); curFunctionCosts = new double[costFunctions.size()]; @@ -452,7 +460,8 @@ protected List balanceTable(TableName tableName, // Allow turning this feature off if the locality cost is not going to // be used in any computations. RegionHDFSBlockLocationFinder finder = null; - if ((this.localityCost != null) || (this.rackLocalityCost != null)) { + if ((this.localityCost != null) || (this.rackLocalityCost != null) || + (this.prefetchCacheCost != null)) { finder = this.regionFinder; } 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..6993f2a24401 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 @@ -66,6 +66,63 @@ public class TestStochasticLoadBalancer extends StochasticBalancerTestBase { private static final String REGION_KEY = "testRegion"; + // Mapping of prefetch test -> expected prefetch + private float[] expectedPrefetch = { 0.0f, 1.0f, 0.5f, 0.75f, 0.0f}; + + /** + * 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 int[][][] clusterRegionPrefetchMocks = new int[][][] { + // Test 1: each region is entirely on server that hosts it + new int[][] { + new int[] { 2, 1, 1}, + new int[] { 2, 0, 0, 100 }, // region 0 is hosted and entirely prefethced on server 2 + new int[] { 0, 100, 0, 0 }, // region 1 is hosted and entirely prefetched on server 0 + new int[] { 0, 100, 0, 0 }, // region 2 is hosted and entirely prefetched on server 0 + new int[] { 1, 0, 100, 0 }, // region 3 is hosted and entirely prefetched on server 1 + }, + + // Test 2: each region is 0% local on the server that hosts it + new int[][] { + new int[] { 1, 2, 1 }, + new int[] { 0, 0, 0, 100 }, // region 0 is hosted and entirely prefetched on server 2 + new int[] { 1, 100, 0, 0 }, // region 1 is hosted and prefetched entirely on server 0 + new int[] { 1, 100, 0, 0 }, // region 2 is hosted and prefetched entirely on server 0 + new int[] { 2, 0, 100, 0 }, // region 3 is hosted and prefetched entirely on server 1 + }, + + // Test 3: each region is 25% prefetched on the server that hosts it while 50% prefetched + // on some other server + new int[][] { + new int[] { 1, 2, 1 }, + new int[] { 0, 25, 0, 50 }, + new int[] { 1, 50, 25, 0 }, + new int[] { 1, 50, 25, 0 }, + new int[] { 2, 0, 50, 25 }, + }, + + // Test 4: each region is 25% prefetched on the server that hosts it and 100% prefetched + // on some other server + new int[][] { + new int[] { 1, 2, 1 }, + new int[] { 0, 25, 0, 100 }, + new int[] { 1, 100, 25, 0 }, + new int[] { 1, 100, 25, 0 }, + new int[] { 2, 0, 100, 25 }, + }, + + // Test 5: each region is 75% prefetched on all the servers + new int[][] { + new int[] { 1, 2, 1 }, + new int[] { 0, 75, 75, 75 }, + new int[] { 1, 75, 75, 75 }, + new int[] { 1, 75, 75, 75 }, + new int[] { 2, 75, 75, 75 }, + }, + }; + // Mapping of locality test -> expected locality private float[] expectedLocalities = { 1.0f, 0.0f, 0.50f, 0.25f, 1.0f }; private static Configuration storedConfiguration; @@ -213,6 +270,7 @@ 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.getPrefetchCacheRatio()).thenReturn(0.0f); Map regionLoadMap = new TreeMap<>(Bytes.BYTES_COMPARATOR); regionLoadMap.put(Bytes.toBytes(REGION_KEY), rl); @@ -623,6 +681,70 @@ private boolean needsBalanceIdleRegion(int[] cluster) { && Arrays.stream(cluster).anyMatch(x -> x < 1); } + @Test + public void testVerifyPrefetchCostFunctionEnabled() { + conf.set("hbase.prefetch.file-list.path", "/tmp/prefetch.persistence"); + + StochasticLoadBalancer lb = new StochasticLoadBalancer(); + lb.loadConf(conf); + + assertTrue(Arrays.asList(lb.getCostFunctionNames()) + .contains(PrefetchCacheCostFunction.class.getSimpleName())); + } + + @Test + public void testVerifyPrefetchCostFunctionNotEnabled() { + assertFalse(Arrays.asList(loadBalancer.getCostFunctionNames()) + .contains(PrefetchCacheCostFunction.class.getSimpleName())); + } + + @Test + public void testPrefetchCost() throws Exception { + conf.set("hbase.prefetch.file-list.path", "/tmp/prefetch.persistence"); + CostFunction costFunction = new PrefetchCacheCostFunction(conf); + + for (int test = 0; test < clusterRegionPrefetchMocks.length; test++) { + int[][] clusterRegionLocations = clusterRegionPrefetchMocks[test]; + MockClusterForPrefetch cluster = new MockClusterForPrefetch(clusterRegionLocations); + costFunction.prepare(cluster); + double cost = costFunction.cost(); + assertEquals(expectedPrefetch[test], cost, 0.01); + } + } + + private class MockClusterForPrefetch extends BalancerClusterState { + private int[][] regionServerPrefetch = null; // [region][server] = prefetch percent + public MockClusterForPrefetch(int[][] regions) { + // regions[0] is an array where index = serverIndex and value = number of regions + super(mockClusterServers(regions[0], 1), null, null, null); + regionServerPrefetch = new int[regions.length - 1][]; + for (int i = 1; i < regions.length; i++) { + int regionIndex = i - 1; + regionServerPrefetch[regionIndex] = new int[regions[i].length - 1]; + regionIndexToServerIndex[regionIndex] = regions[i][0]; + for (int j = 1; j < regions[i].length; j++) { + int serverIndex = j - 1; + regionServerPrefetch[regionIndex][serverIndex] = regions[i][j]; + } + } + } + + @Override + public float getOrComputeWeightedPrefetchRatio(int region, int server) { + return getRegionSizeMB(region) * regionServerPrefetch[region][server]/100.0f; + } + + @Override + public int getRegionSizeMB(int region) { + return 1; + } + + @Override + protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) { + return regionServerPrefetch[region][regionServerIndex]/100.0f; + } + } + // This mock allows us to test the LocalityCostFunction private class MockCluster extends BalancerClusterState { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java index 47b36a7a1516..f07650147b4f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java @@ -138,4 +138,7 @@ default String getNameAsString() { /** Returns the compaction state of this region */ CompactionState getCompactionState(); + + /** Returns the ratio of files already in cache */ + float getPrefetchCacheRatio(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java index 43b3a17aac17..0c0e504ed6dc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java @@ -80,6 +80,7 @@ public static RegionMetrics toRegionMetrics(ClusterStatusProtos.RegionLoad regio ClusterStatusProtos.StoreSequenceId::getSequenceId))) .setUncompressedStoreFileSize( new Size(regionLoadPB.getStoreUncompressedSizeMB(), Size.Unit.MEGABYTE)) + .setPrefetchCacheRatio(regionLoadPB.getPrefetchCacheRatio()) .build(); } @@ -120,6 +121,7 @@ public static ClusterStatusProtos.RegionLoad toRegionLoad(RegionMetrics regionMe .addAllStoreCompleteSequenceId(toStoreSequenceId(regionMetrics.getStoreSequenceId())) .setStoreUncompressedSizeMB( (int) regionMetrics.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE)) + .setPrefetchCacheRatio(regionMetrics.getPrefetchCacheRatio()) .build(); } @@ -154,6 +156,7 @@ public static RegionMetricsBuilder newBuilder(byte[] name) { private long blocksLocalWithSsdWeight; private long blocksTotalWeight; private CompactionState compactionState; + private float prefetchCacheRatio; private RegionMetricsBuilder(byte[] name) { this.name = name; @@ -289,6 +292,11 @@ public RegionMetricsBuilder setCompactionState(CompactionState compactionState) return this; } + public RegionMetricsBuilder setPrefetchCacheRatio(float prefetchCacheRatio) { + this.prefetchCacheRatio = prefetchCacheRatio; + return this; + } + public RegionMetrics build() { return new RegionMetricsImpl(name, storeCount, storeFileCount, storeRefCount, maxCompactedStoreFileRefCount, compactingCellCount, compactedCellCount, storeFileSize, @@ -296,7 +304,7 @@ public RegionMetrics build() { uncompressedStoreFileSize, writeRequestCount, readRequestCount, cpRequestCount, filteredReadRequestCount, completedSequenceId, storeSequenceIds, dataLocality, lastMajorCompactionTimestamp, dataLocalityForSsd, blocksLocalWeight, blocksLocalWithSsdWeight, - blocksTotalWeight, compactionState); + blocksTotalWeight, compactionState, prefetchCacheRatio); } private static class RegionMetricsImpl implements RegionMetrics { @@ -327,6 +335,7 @@ private static class RegionMetricsImpl implements RegionMetrics { private final long blocksLocalWithSsdWeight; private final long blocksTotalWeight; private final CompactionState compactionState; + private final float prefetchCacheRatio; RegionMetricsImpl(byte[] name, int storeCount, int storeFileCount, int storeRefCount, int maxCompactedStoreFileRefCount, final long compactingCellCount, long compactedCellCount, @@ -336,7 +345,7 @@ private static class RegionMetricsImpl implements RegionMetrics { long filteredReadRequestCount, long completedSequenceId, Map storeSequenceIds, float dataLocality, long lastMajorCompactionTimestamp, float dataLocalityForSsd, long blocksLocalWeight, long blocksLocalWithSsdWeight, long blocksTotalWeight, - CompactionState compactionState) { + CompactionState compactionState, float prefetchCacheRatio) { this.name = Preconditions.checkNotNull(name); this.storeCount = storeCount; this.storeFileCount = storeFileCount; @@ -364,6 +373,7 @@ private static class RegionMetricsImpl implements RegionMetrics { this.blocksLocalWithSsdWeight = blocksLocalWithSsdWeight; this.blocksTotalWeight = blocksTotalWeight; this.compactionState = compactionState; + this.prefetchCacheRatio = prefetchCacheRatio; } @Override @@ -501,6 +511,11 @@ public CompactionState getCompactionState() { return compactionState; } + @Override + public float getPrefetchCacheRatio() { + return prefetchCacheRatio; + } + @Override public String toString() { StringBuilder sb = @@ -541,6 +556,7 @@ public String toString() { Strings.appendKeyValue(sb, "blocksLocalWithSsdWeight", blocksLocalWithSsdWeight); Strings.appendKeyValue(sb, "blocksTotalWeight", blocksTotalWeight); Strings.appendKeyValue(sb, "compactionState", compactionState); + Strings.appendKeyValue(sb, "prefetchCacheRatio", this.getPrefetchCacheRatio()); return sb.toString(); } } diff --git a/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto b/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto index 28cc5a865c23..938f41efc30a 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto @@ -177,6 +177,9 @@ message RegionLoad { MAJOR = 2; MAJOR_AND_MINOR = 3; } + + /** The prefetch cache ratio for region */ + optional float prefetch_cache_ratio = 28; } message UserLoad { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index c18b7e73cdf6..bfeec4a7ffc4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -100,6 +100,7 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory; import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.PrefetchExecutor; import org.apache.hadoop.hbase.io.util.MemorySizeUtil; import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; import org.apache.hadoop.hbase.ipc.RpcClient; @@ -1493,6 +1494,7 @@ RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, long totalStaticBloomSize = 0L; long totalCompactingKVs = 0L; long currentCompactedKVs = 0L; + int filesAlreadyPrefetched = 0; List storeList = r.getStores(); stores += storeList.size(); for (HStore store : storeList) { @@ -1514,6 +1516,14 @@ RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, rootLevelIndexSize += store.getStorefilesRootLevelIndexSize(); totalStaticIndexSize += store.getTotalStaticIndexSize(); totalStaticBloomSize += store.getTotalStaticBloomSize(); + Collection filesInStore = store.getStorefiles(); + if (!filesInStore.isEmpty()) { + for (HStoreFile hStoreFile : filesInStore) { + if (PrefetchExecutor.isFilePrefetched(hStoreFile.getPath().getName())) { + filesAlreadyPrefetched++; + } + } + } } int unitMB = 1024 * 1024; @@ -1533,6 +1543,7 @@ RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, long blocksTotalWeight = hdfsBd.getUniqueBlocksTotalWeight(); long blocksLocalWeight = hdfsBd.getBlocksLocalWeight(serverName.getHostname()); long blocksLocalWithSsdWeight = hdfsBd.getBlocksLocalWithSsdWeight(serverName.getHostname()); + float ratioOfFilesAlreadyCached = filesAlreadyPrefetched/storefiles; if (regionLoadBldr == null) { regionLoadBldr = RegionLoad.newBuilder(); } @@ -1556,7 +1567,8 @@ RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, .setDataLocalityForSsd(dataLocalityForSsd).setBlocksLocalWeight(blocksLocalWeight) .setBlocksLocalWithSsdWeight(blocksLocalWithSsdWeight).setBlocksTotalWeight(blocksTotalWeight) .setCompactionState(ProtobufUtil.createCompactionStateForRegionLoad(r.getCompactionState())) - .setLastMajorCompactionTs(r.getOldestHfileTs(true)); + .setLastMajorCompactionTs(r.getOldestHfileTs(true)) + .setPrefetchCacheRatio(ratioOfFilesAlreadyCached); r.setCompleteSequenceId(regionLoadBldr); return regionLoadBldr.build(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java index 53633bd774f6..a2abcc665579 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java @@ -541,6 +541,9 @@ public long getBlocksTotalWeight() { public CompactionState getCompactionState() { return null; } + + @Override + public float getPrefetchCacheRatio() { return 0.0f; } }; return regionMetrics; } From 54bacfff712532c269e241b791e87c9282aad22b Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Sat, 24 Sep 2022 01:37:46 +0530 Subject: [PATCH 02/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions --- .../master/balancer/BalancerClusterState.java | 4 +- .../master/balancer/BalancerRegionLoad.java | 4 +- .../PrefetchBasedCandidateGenerator.java | 15 +++--- .../balancer/PrefetchCacheCostFunction.java | 15 +++--- .../balancer/StochasticLoadBalancer.java | 7 +-- .../balancer/TestStochasticLoadBalancer.java | 49 +++++++------------ .../hadoop/hbase/RegionMetricsBuilder.java | 6 +-- .../hbase/regionserver/HRegionServer.java | 5 +- .../master/TestRegionsRecoveryChore.java | 4 +- 9 files changed, 47 insertions(+), 62 deletions(-) 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 dcc4ea96895e..8c079cbf30a9 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 @@ -606,8 +606,8 @@ protected float getRegionServerPrefetchRatio(int region, int server) { Deque regionLoadList = regionLoads[regionIndex]; // Found the region load. Find the prefetch ratio for this region load - prefetchRatio = regionLoadList.isEmpty() ? 0.0f : - regionLoadList.getLast().getPrefetchCacheRatio(); + prefetchRatio = + regionLoadList == null ? 0.0f : regionLoadList.getLast().getPrefetchCacheRatio(); break; } return prefetchRatio; 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 25dff972f526..ce51543089de 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 @@ -65,5 +65,7 @@ public int getStorefileSizeMB() { return storefileSizeMB; } - public float getPrefetchCacheRatio() { return prefetchCacheRatio; } + public float getPrefetchCacheRatio() { + return prefetchCacheRatio; + } } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java index ee5125a57675..fc56255c5058 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.balancer; import java.util.Optional; @@ -23,7 +22,7 @@ import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private -class PrefetchBasedCandidateGenerator extends CandidateGenerator{ +class PrefetchBasedCandidateGenerator extends CandidateGenerator { @Override BalanceAction generate(BalancerClusterState cluster) { // iterate through regions until you find one that is not on ideal host @@ -33,11 +32,9 @@ BalanceAction generate(BalancerClusterState cluster) { for (int i = 0; i < cluster.numRegions; i++) { int region = (startIndex + i) % cluster.numRegions; int currentServer = cluster.regionIndexToServerIndex[region]; - if ( - currentServer != cluster.getOrComputeServerWithBestPrefetchRatio()[region] - ) { - Optional potential = tryMoveOrSwap(cluster, - currentServer, region, cluster.getOrComputeServerWithBestPrefetchRatio()[region]); + if (currentServer != cluster.getOrComputeServerWithBestPrefetchRatio()[region]) { + Optional potential = tryMoveOrSwap(cluster, currentServer, region, + cluster.getOrComputeServerWithBestPrefetchRatio()[region]); if (potential.isPresent()) { return potential.get(); } @@ -47,8 +44,8 @@ BalanceAction generate(BalancerClusterState cluster) { return BalanceAction.NULL_ACTION; } - private Optional tryMoveOrSwap(BalancerClusterState cluster, - int fromServer, int fromRegion, int toServer) { + private Optional tryMoveOrSwap(BalancerClusterState cluster, int fromServer, + int fromRegion, int toServer) { // Try move first. We know apriori fromRegion has the highest locality on toServer if (cluster.serverHasTooFewRegions(toServer)) { return Optional.of(getAction(fromServer, fromRegion, toServer, -1)); diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java index 3738f48f44ff..14397da75c06 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -15,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.balancer; import org.apache.hadoop.conf.Configuration; import org.apache.yetus.audience.InterfaceAudience; /** - * Compute the cost of a potential cluster configuration based on - * the number of HFile's already cached in the bucket cache + * Compute the cost of a potential cluster configuration based on the number of HFile's already + * cached in the bucket cache */ @InterfaceAudience.Private public class PrefetchCacheCostFunction extends CostFunction { @@ -47,13 +46,12 @@ void prepare(BalancerClusterState cluster) { bestPrefetchRatio = 0.0f; for (int region = 0; region < cluster.numRegions; region++) { - prefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region, - cluster.regionIndexToServerIndex[region]); + prefetchRatio += + cluster.getOrComputeWeightedPrefetchRatio(region, cluster.regionIndexToServerIndex[region]); bestPrefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region, cluster.getOrComputeServerWithBestPrefetchRatio()[region]); } - prefetchRatio = bestPrefetchRatio == 0.0f ? 1.0f : - prefetchRatio/bestPrefetchRatio; + prefetchRatio = bestPrefetchRatio == 0.0f ? 1.0f : prefetchRatio / bestPrefetchRatio; } @Override @@ -66,8 +64,7 @@ protected void regionMoved(int region, int oldServer, int newServer) { float oldServerPrefetch = getWeightedPrefetchRatio(region, oldServer); float newServerPrefetch = getWeightedPrefetchRatio(region, newServer); float prefetchDelta = newServerPrefetch - oldServerPrefetch; - float normalizeDelta = bestPrefetchRatio == 0.0f ? 0.0f : - prefetchDelta/bestPrefetchRatio; + float normalizeDelta = bestPrefetchRatio == 0.0f ? 0.0f : prefetchDelta / bestPrefetchRatio; prefetchRatio += normalizeDelta; } 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 f8f970d9cced..8bea0a7f1f8e 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 @@ -156,7 +156,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { private PrefetchCacheCostFunction prefetchCacheCost; private PrefetchBasedCandidateGenerator prefetchCandidateGenerator; - protected List candidateGenerators; public enum GeneratorType { @@ -460,8 +459,10 @@ protected List balanceTable(TableName tableName, // Allow turning this feature off if the locality cost is not going to // be used in any computations. RegionHDFSBlockLocationFinder finder = null; - if ((this.localityCost != null) || (this.rackLocalityCost != null) || - (this.prefetchCacheCost != null)) { + if ( + (this.localityCost != null) || (this.rackLocalityCost != null) + || (this.prefetchCacheCost != null) + ) { finder = this.regionFinder; } 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 6993f2a24401..748e4814fd3b 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 @@ -67,7 +67,7 @@ public class TestStochasticLoadBalancer extends StochasticBalancerTestBase { private static final String REGION_KEY = "testRegion"; // Mapping of prefetch test -> expected prefetch - private float[] expectedPrefetch = { 0.0f, 1.0f, 0.5f, 0.75f, 0.0f}; + private float[] expectedPrefetch = { 0.0f, 1.0f, 0.5f, 0.75f, 0.0f }; /** * Data set to testPrefetchCost: [test][0][0] = mapping of server to number of regions it hosts @@ -76,18 +76,18 @@ public class TestStochasticLoadBalancer extends StochasticBalancerTestBase { */ private int[][][] clusterRegionPrefetchMocks = new int[][][] { // Test 1: each region is entirely on server that hosts it - new int[][] { - new int[] { 2, 1, 1}, - new int[] { 2, 0, 0, 100 }, // region 0 is hosted and entirely prefethced on server 2 + new int[][] { new int[] { 2, 1, 1 }, new int[] { 2, 0, 0, 100 }, // region 0 is hosted and + // entirely prefethced on + // server 2 new int[] { 0, 100, 0, 0 }, // region 1 is hosted and entirely prefetched on server 0 new int[] { 0, 100, 0, 0 }, // region 2 is hosted and entirely prefetched on server 0 new int[] { 1, 0, 100, 0 }, // region 3 is hosted and entirely prefetched on server 1 }, // Test 2: each region is 0% local on the server that hosts it - new int[][] { - new int[] { 1, 2, 1 }, - new int[] { 0, 0, 0, 100 }, // region 0 is hosted and entirely prefetched on server 2 + new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 0, 0, 100 }, // region 0 is hosted and + // entirely prefetched on + // server 2 new int[] { 1, 100, 0, 0 }, // region 1 is hosted and prefetched entirely on server 0 new int[] { 1, 100, 0, 0 }, // region 2 is hosted and prefetched entirely on server 0 new int[] { 2, 0, 100, 0 }, // region 3 is hosted and prefetched entirely on server 1 @@ -95,33 +95,17 @@ public class TestStochasticLoadBalancer extends StochasticBalancerTestBase { // Test 3: each region is 25% prefetched on the server that hosts it while 50% prefetched // on some other server - new int[][] { - new int[] { 1, 2, 1 }, - new int[] { 0, 25, 0, 50 }, - new int[] { 1, 50, 25, 0 }, - new int[] { 1, 50, 25, 0 }, - new int[] { 2, 0, 50, 25 }, - }, + new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 25, 0, 50 }, new int[] { 1, 50, 25, 0 }, + new int[] { 1, 50, 25, 0 }, new int[] { 2, 0, 50, 25 }, }, // Test 4: each region is 25% prefetched on the server that hosts it and 100% prefetched // on some other server - new int[][] { - new int[] { 1, 2, 1 }, - new int[] { 0, 25, 0, 100 }, - new int[] { 1, 100, 25, 0 }, - new int[] { 1, 100, 25, 0 }, - new int[] { 2, 0, 100, 25 }, - }, + new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 25, 0, 100 }, new int[] { 1, 100, 25, 0 }, + new int[] { 1, 100, 25, 0 }, new int[] { 2, 0, 100, 25 }, }, // Test 5: each region is 75% prefetched on all the servers - new int[][] { - new int[] { 1, 2, 1 }, - new int[] { 0, 75, 75, 75 }, - new int[] { 1, 75, 75, 75 }, - new int[] { 1, 75, 75, 75 }, - new int[] { 2, 75, 75, 75 }, - }, - }; + new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 75, 75, 75 }, new int[] { 1, 75, 75, 75 }, + new int[] { 1, 75, 75, 75 }, new int[] { 2, 75, 75, 75 }, }, }; // Mapping of locality test -> expected locality private float[] expectedLocalities = { 1.0f, 0.0f, 0.50f, 0.25f, 1.0f }; @@ -714,8 +698,9 @@ public void testPrefetchCost() throws Exception { private class MockClusterForPrefetch extends BalancerClusterState { private int[][] regionServerPrefetch = null; // [region][server] = prefetch percent + public MockClusterForPrefetch(int[][] regions) { - // regions[0] is an array where index = serverIndex and value = number of regions + // regions[0] is an array where index = serverIndex and value = number of regions super(mockClusterServers(regions[0], 1), null, null, null); regionServerPrefetch = new int[regions.length - 1][]; for (int i = 1; i < regions.length; i++) { @@ -731,7 +716,7 @@ public MockClusterForPrefetch(int[][] regions) { @Override public float getOrComputeWeightedPrefetchRatio(int region, int server) { - return getRegionSizeMB(region) * regionServerPrefetch[region][server]/100.0f; + return getRegionSizeMB(region) * regionServerPrefetch[region][server] / 100.0f; } @Override @@ -741,7 +726,7 @@ public int getRegionSizeMB(int region) { @Override protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) { - return regionServerPrefetch[region][regionServerIndex]/100.0f; + return regionServerPrefetch[region][regionServerIndex] / 100.0f; } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java index 0c0e504ed6dc..d04534cf0662 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java @@ -80,8 +80,7 @@ public static RegionMetrics toRegionMetrics(ClusterStatusProtos.RegionLoad regio ClusterStatusProtos.StoreSequenceId::getSequenceId))) .setUncompressedStoreFileSize( new Size(regionLoadPB.getStoreUncompressedSizeMB(), Size.Unit.MEGABYTE)) - .setPrefetchCacheRatio(regionLoadPB.getPrefetchCacheRatio()) - .build(); + .setPrefetchCacheRatio(regionLoadPB.getPrefetchCacheRatio()).build(); } private static List @@ -121,8 +120,7 @@ public static ClusterStatusProtos.RegionLoad toRegionLoad(RegionMetrics regionMe .addAllStoreCompleteSequenceId(toStoreSequenceId(regionMetrics.getStoreSequenceId())) .setStoreUncompressedSizeMB( (int) regionMetrics.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE)) - .setPrefetchCacheRatio(regionMetrics.getPrefetchCacheRatio()) - .build(); + .setPrefetchCacheRatio(regionMetrics.getPrefetchCacheRatio()).build(); } public static RegionMetricsBuilder newBuilder(byte[] name) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index bfeec4a7ffc4..626fb3a5cae7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1543,7 +1543,10 @@ RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, long blocksTotalWeight = hdfsBd.getUniqueBlocksTotalWeight(); long blocksLocalWeight = hdfsBd.getBlocksLocalWeight(serverName.getHostname()); long blocksLocalWithSsdWeight = hdfsBd.getBlocksLocalWithSsdWeight(serverName.getHostname()); - float ratioOfFilesAlreadyCached = filesAlreadyPrefetched/storefiles; + + float ratioOfFilesAlreadyCached = + (storefiles == 0) ? 0 : filesAlreadyPrefetched / ((float) storefiles); + if (regionLoadBldr == null) { regionLoadBldr = RegionLoad.newBuilder(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java index a2abcc665579..28f92ac319dc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java @@ -543,7 +543,9 @@ public CompactionState getCompactionState() { } @Override - public float getPrefetchCacheRatio() { return 0.0f; } + public float getPrefetchCacheRatio() { + return 0.0f; + } }; return regionMetrics; } From 5c201eda88d99c6e7095e5075230a8b0509f1baf Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Wed, 18 Jan 2023 15:18:29 +0530 Subject: [PATCH 03/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions --- .../master/balancer/BalancerClusterState.java | 113 +++++--- .../master/balancer/BalancerRegionLoad.java | 4 + .../master/balancer/BaseLoadBalancer.java | 2 +- .../PrefetchBasedCandidateGenerator.java | 61 ++--- .../balancer/PrefetchCacheCostFunction.java | 24 +- .../balancer/StochasticLoadBalancer.java | 30 +- .../LoadBalancerPerformanceEvaluation.java | 1 + ...PrefetchCacheCostLoadBalancerFunction.java | 258 ++++++++++++++++++ .../balancer/TestStochasticLoadBalancer.java | 113 +------- ...tStochasticLoadBalancerBalanceCluster.java | 2 + ...estStochasticLoadBalancerLargeCluster.java | 2 + ...dBalancerRegionReplicaHighReplication.java | 2 + ...estStochasticLoadBalancerSmallCluster.java | 4 + .../apache/hadoop/hbase/RegionMetrics.java | 3 + .../hadoop/hbase/RegionMetricsBuilder.java | 24 +- .../org/apache/hadoop/hbase/HConstants.java | 5 + .../main/protobuf/server/ClusterStatus.proto | 3 + .../hadoop/hbase/io/hfile/CacheConfig.java | 2 - .../hbase/io/hfile/bucket/BucketCache.java | 5 +- .../hbase/regionserver/HRegionServer.java | 3 +- .../hbase/io/hfile/TestPrefetchRSClose.java | 3 +- .../hfile/bucket/TestPrefetchPersistence.java | 3 +- .../master/TestRegionsRecoveryChore.java | 3 + .../balancer/TestBalancerRejection.java | 2 + 24 files changed, 466 insertions(+), 206 deletions(-) create mode 100644 hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.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 8c079cbf30a9..6787c584d950 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 @@ -116,9 +116,11 @@ class BalancerClusterState { private int[][] regionsToMostLocalEntities; // Maps region -> serverIndex -> prefetch ratio of a region on a server - private float[][] regionServerPrefetchRatio; + private Map, Float> regionIndexServerIndexPrefetchRatio; // Maps region -> serverIndex with best prefect ratio - private int[] regionServerWithBestPrefetchRatio; + private int[] regionServerIndexWithBestPrefetchRatio; + // Historical region server prefetch ratio + Map> historicalRegionServerPrefetchRatio; static class DefaultRackManager extends RackManager { @Override @@ -130,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); + } + + 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(); } @@ -150,6 +159,8 @@ public String getRack(ServerName server) { tables = new ArrayList<>(); this.rackManager = rackManager != null ? rackManager : new DefaultRackManager(); + this.historicalRegionServerPrefetchRatio = oldRegionServerPrefetchRatio; + numRegions = 0; List> serversPerHostList = new ArrayList<>(); @@ -559,58 +570,98 @@ enum LocalityType { } public float getOrComputeWeightedPrefetchRatio(int region, int server) { - return getRegionSizeMB(region) * getOrComputeRegionPrefetchRatio()[region][server]; + return getRegionSizeMB(region) * getOrComputeRegionPrefetchRatio(region, server); } - private float[][] getOrComputeRegionPrefetchRatio() { - if (regionServerWithBestPrefetchRatio == null || regionServerPrefetchRatio == null) { - computeRegionServerPrefetchRatio(); + protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) { + // Cost this server has from RegionLoad + float prefetchRatio = 0.0f; + + // Get the prefetch ratio if the region is currently hosted on this server + for (int regionIndex : regionsPerServer[regionServerIndex]) { + if (region != regionIndex) { + continue; + } + Deque regionLoadList = regionLoads[regionIndex]; + + // The region is currently hosted on this region server. Now, get the prefetch cache ratio + // for this region on this region server + prefetchRatio = regionLoadList == null ? 0.0f : regionLoadList.getLast() + .getPrefetchCacheRatio(); + + return prefetchRatio; } - return regionServerPrefetchRatio; - } - public int[] getOrComputeServerWithBestPrefetchRatio() { - if (regionServerWithBestPrefetchRatio == null || regionServerPrefetchRatio == null) { - computeRegionServerPrefetchRatio(); + // 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. + // Seartch 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 regionNameAsString = regions[region].getRegionNameAsString(); + String serverNameAsString = servers[regionServerIndex].getServerName(); + if (historicalRegionServerPrefetchRatio != null && + historicalRegionServerPrefetchRatio.containsKey(regionNameAsString)) { + Map serverPrefetchRatio = + historicalRegionServerPrefetchRatio.get(regionNameAsString); + if (serverPrefetchRatio.containsKey(serverNameAsString)) { + prefetchRatio = serverPrefetchRatio.get(serverNameAsString); + + // The old prefetch cache ratio has been accounted for and hence, clear up this information + historicalRegionServerPrefetchRatio.remove(regionNameAsString, serverPrefetchRatio); + } } - return regionServerWithBestPrefetchRatio; + return prefetchRatio; } private void computeRegionServerPrefetchRatio() { - regionServerPrefetchRatio = new float[numRegions][numServers]; - regionServerWithBestPrefetchRatio = new int[numRegions]; + 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); - regionServerPrefetchRatio[region][server] = prefetchRatio; + if (prefetchRatio > 0.0f || server == regionIndexToServerIndex[region]) { + // A region with prefetch ratio of 0 on a server means nothing. Hence, just make a note + // of prefetch only if the prefetch ratio is greater than 0. + Map tempMap = new HashMap<>(); + tempMap.put(region, server); + regionIndexServerIndexPrefetchRatio.put(tempMap, 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 the same prefetch ratio, the the server currently hostring the + // region should retain the region + serverWithBestPrefetchRatio = server; } } - regionServerWithBestPrefetchRatio[region] = serverWithBestPrefetchRatio; + regionServerIndexWithBestPrefetchRatio[region] = serverWithBestPrefetchRatio; } } - protected float getRegionServerPrefetchRatio(int region, int server) { - float prefetchRatio = 0.0f; + private float getOrComputeRegionPrefetchRatio(int region, int server) { + if (regionServerIndexWithBestPrefetchRatio == null || + regionIndexServerIndexPrefetchRatio.isEmpty()) { + computeRegionServerPrefetchRatio(); + } - // For every region on this region server get the region load - for (int regionIndex : regionsPerServer[server]) { - if (region != regionIndex) { - continue; - } - Deque regionLoadList = regionLoads[regionIndex]; + Map tempMap = new HashMap<>(); + tempMap.put(region, server); + return regionIndexServerIndexPrefetchRatio.containsKey(tempMap) ? + regionIndexServerIndexPrefetchRatio.get(tempMap) : 0.0f; + } - // Found the region load. Find the prefetch ratio for this region load - prefetchRatio = - regionLoadList == null ? 0.0f : regionLoadList.getLast().getPrefetchCacheRatio(); - break; + public int[] getOrComputeServerWithBestPrefetchRatio() { + if (regionIndexServerIndexPrefetchRatio.isEmpty() || + regionServerIndexWithBestPrefetchRatio == null) { + computeRegionServerPrefetchRatio(); } - return prefetchRatio; + return regionServerIndexWithBestPrefetchRatio; } public void doAction(BalanceAction action) { 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 ce51543089de..20d2e764221f 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 @@ -35,6 +35,7 @@ class BalancerRegionLoad { private final int memStoreSizeMB; private final int storefileSizeMB; private final float prefetchCacheRatio; + private final String serverName; BalancerRegionLoad(RegionMetrics regionMetrics) { readRequestsCount = regionMetrics.getReadRequestCount(); @@ -43,6 +44,7 @@ class BalancerRegionLoad { memStoreSizeMB = (int) regionMetrics.getMemStoreSize().get(Size.Unit.MEGABYTE); storefileSizeMB = (int) regionMetrics.getStoreFileSize().get(Size.Unit.MEGABYTE); prefetchCacheRatio = regionMetrics.getPrefetchCacheRatio(); + serverName = regionMetrics.getServerName(); } public long getReadRequestsCount() { @@ -68,4 +70,6 @@ public int getStorefileSizeMB() { public float getPrefetchCacheRatio() { return prefetchCacheRatio; } + + public String getServerName() { return serverName; } } 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 b6abdd5c7f6e..f8ca647e50f1 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,7 @@ 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/PrefetchBasedCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java index fc56255c5058..6e75fcce52ef 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java @@ -20,57 +20,40 @@ import java.util.Optional; import java.util.concurrent.ThreadLocalRandom; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @InterfaceAudience.Private class PrefetchBasedCandidateGenerator extends CandidateGenerator { + private static final Logger LOG = LoggerFactory.getLogger(PrefetchBasedCandidateGenerator.class); + + private static float PREFETCH_RATIO_DIFF_FACTOR = 1.25f; + @Override BalanceAction generate(BalancerClusterState cluster) { // iterate through regions until you find one that is not on ideal host // start from a random point to avoid always balance the regions in front if (cluster.numRegions > 0) { - int startIndex = ThreadLocalRandom.current().nextInt(cluster.numRegions); + int startRegionIndex = ThreadLocalRandom.current().nextInt(cluster.numRegions); + int toServerIndex; for (int i = 0; i < cluster.numRegions; i++) { - int region = (startIndex + i) % cluster.numRegions; - int currentServer = cluster.regionIndexToServerIndex[region]; - if (currentServer != cluster.getOrComputeServerWithBestPrefetchRatio()[region]) { - Optional potential = tryMoveOrSwap(cluster, currentServer, region, - cluster.getOrComputeServerWithBestPrefetchRatio()[region]); - if (potential.isPresent()) { - return potential.get(); - } - } - } - } - return BalanceAction.NULL_ACTION; - } + int region = (startRegionIndex + i) % cluster.numRegions; + int currentServerIndex = cluster.regionIndexToServerIndex[region]; + float currentPrefetchRatio = + cluster.getOrComputeWeightedPrefetchRatio(region, currentServerIndex); - private Optional tryMoveOrSwap(BalancerClusterState cluster, int fromServer, - int fromRegion, int toServer) { - // Try move first. We know apriori fromRegion has the highest locality on toServer - if (cluster.serverHasTooFewRegions(toServer)) { - return Optional.of(getAction(fromServer, fromRegion, toServer, -1)); - } - // Compare prefetch gain/loss from swapping fromRegion with regions on toServer - float fromRegionPrefetchDelta = getWeightedPrefetch(cluster, fromRegion, toServer) - - getWeightedPrefetch(cluster, fromRegion, fromServer); - int toServertotalRegions = cluster.regionsPerServer[toServer].length; - if (toServertotalRegions > 0) { - int startIndex = ThreadLocalRandom.current().nextInt(toServertotalRegions); - for (int i = 0; i < toServertotalRegions; i++) { - int toRegionIndex = (startIndex + i) % toServertotalRegions; - int toRegion = cluster.regionsPerServer[toServer][toRegionIndex]; - float toRegionPrefetchDelta = getWeightedPrefetch(cluster, toRegion, fromServer) - - getWeightedPrefetch(cluster, toRegion, toServer); - // If prefetch would remain neutral or improve, attempt the swap - if (fromRegionPrefetchDelta + toRegionPrefetchDelta >= 0) { - return Optional.of(getAction(fromServer, fromRegion, toServer, toRegion)); + // Check if there is a server with a better historical prefetch ratio + toServerIndex = pickOtherRandomServer(cluster, currentServerIndex); + float toServerPrefetchRatio = + cluster.getOrComputeWeightedPrefetchRatio(region, toServerIndex); + + // If the prefetch ratio on the target server is significantly higher, move the region. + if (currentPrefetchRatio > 0 && + (toServerPrefetchRatio / currentPrefetchRatio) > PREFETCH_RATIO_DIFF_FACTOR) { + return getAction(currentServerIndex, region, toServerIndex, -1); } } } - return Optional.empty(); - } - - private float getWeightedPrefetch(BalancerClusterState cluster, int region, int server) { - return cluster.getOrComputeWeightedPrefetchRatio(region, server); + return BalanceAction.NULL_ACTION; } } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java index 14397da75c06..ee66c6971e39 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.balancer; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; /** @@ -26,15 +27,20 @@ */ @InterfaceAudience.Private public class PrefetchCacheCostFunction extends CostFunction { + private static final String PREFETCH_CACHE_COST_KEY = + "hbase.master.balancer.stochastic.prefetchCacheCost"; + private static final float DEFAULT_PREFETCH_COST = 500; + private String prefetchedFileListPath; - private float prefetchRatio; + private double prefetchRatio; private float bestPrefetchRatio; - public static final String PREFETCH_PERSISTENCE_PATH_KEY = "hbase.prefetch.file-list.path"; - PrefetchCacheCostFunction(Configuration conf) { - prefetchedFileListPath = conf.get(PREFETCH_PERSISTENCE_PATH_KEY); - this.setMultiplier(prefetchedFileListPath == null ? 0 : 1); + prefetchedFileListPath = conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY); + // Disable the prefetch cache cost function if the prefetched file list persistence is not + // enabled + this.setMultiplier(prefetchedFileListPath == null ? 0.0f : + conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_COST)); prefetchRatio = 0.0f; bestPrefetchRatio = 0.0f; } @@ -61,8 +67,8 @@ protected double cost() { @Override protected void regionMoved(int region, int oldServer, int newServer) { - float oldServerPrefetch = getWeightedPrefetchRatio(region, oldServer); - float newServerPrefetch = getWeightedPrefetchRatio(region, newServer); + float oldServerPrefetch = cluster.getOrComputeWeightedPrefetchRatio(region, oldServer); + float newServerPrefetch = cluster.getOrComputeWeightedPrefetchRatio(region, newServer); float prefetchDelta = newServerPrefetch - oldServerPrefetch; float normalizeDelta = bestPrefetchRatio == 0.0f ? 0.0f : prefetchDelta / bestPrefetchRatio; prefetchRatio += normalizeDelta; @@ -72,8 +78,4 @@ protected void regionMoved(int region, int oldServer, int newServer) { public final void updateWeight(double[] weights) { weights[StochasticLoadBalancer.GeneratorType.PREFETCH.ordinal()] += cost(); } - - private float getWeightedPrefetchRatio(int region, int server) { - return cluster.getOrComputeWeightedPrefetchRatio(region, server); - } } 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 8bea0a7f1f8e..0d568ec0af6f 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 @@ -129,6 +129,10 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { Map> loads = new HashMap<>(); + // Map of old prefetch ratio (region name ---> old server name ---> old prefetch ratio) + Map> historicRegionServerPrefetchRatio = + new HashMap>(); + // values are defaults private int maxSteps = DEFAULT_MAX_STEPS; private boolean runMaxSteps = DEFAULT_RUN_MAX_STEPS; @@ -298,7 +302,8 @@ private void updateBalancerTableLoadInfo(TableName tableName, finder = this.regionFinder; } BalancerClusterState cluster = - new BalancerClusterState(loadOfOneTable, loads, finder, rackManager); + new BalancerClusterState(loadOfOneTable, loads, finder, rackManager, + historicRegionServerPrefetchRatio); initCosts(cluster); curOverallCost = computeCost(cluster, Double.MAX_VALUE); @@ -470,7 +475,8 @@ 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, + historicRegionServerPrefetchRatio); long startTime = EnvironmentEdgeManager.currentTime(); @@ -721,6 +727,9 @@ private List createRegionPlans(BalancerClusterState cluster) { private void updateRegionLoad() { // We create a new hashmap so that regions that are no longer there are removed. // However we temporarily need the old loads so we can use them to keep the rolling average. + // The old prefetch ratio of a region on a region server is stored for finding out if the region + // was prefetched on the old server before moving it to the new server because of the server + // crash procedure. Map> oldLoads = loads; loads = new HashMap<>(); @@ -728,10 +737,23 @@ private void updateRegionLoad() { sm.getRegionMetrics().forEach((byte[] regionName, RegionMetrics rm) -> { String regionNameAsString = RegionInfo.getRegionNameAsString(regionName); Deque rLoads = oldLoads.get(regionNameAsString); + String oldServerName = null; + float oldPrefetchRatio = 0.0f; if (rLoads == null) { rLoads = new ArrayDeque<>(numRegionLoadsToRemember + 1); - } else if (rLoads.size() >= numRegionLoadsToRemember) { - rLoads.remove(); + } else { + // Get the old server name and the prefetch ratio for this region on this server + oldServerName = rLoads.getLast().getServerName(); + oldPrefetchRatio = rLoads.getLast().getPrefetchCacheRatio(); + if (rLoads.size() >= numRegionLoadsToRemember) { + rLoads.remove(); + } + } + if (oldServerName != null && !oldServerName.equals(rm.getServerName())) { + // Record the old region server prefetch ratio + Map serverPrefetchRatio = new HashMap<>(); + serverPrefetchRatio.put(oldServerName, oldPrefetchRatio); + historicRegionServerPrefetchRatio.put(regionNameAsString, serverPrefetchRatio); } rLoads.add(new BalancerRegionLoad(rm)); loads.put(regionNameAsString, rLoads); diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadBalancerPerformanceEvaluation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadBalancerPerformanceEvaluation.java index 3a435e140989..80b3750d6d91 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadBalancerPerformanceEvaluation.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/LoadBalancerPerformanceEvaluation.java @@ -86,6 +86,7 @@ public class LoadBalancerPerformanceEvaluation extends AbstractHBaseTool { private void setupConf() { conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, loadBalancerClazz, LoadBalancer.class); + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch_persistence"); loadBalancer = LoadBalancerFactory.getLoadBalancer(conf); } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java new file mode 100644 index 000000000000..58d8bc76dc83 --- /dev/null +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java @@ -0,0 +1,258 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master.balancer; + +import static org.apache.hadoop.hbase.master.balancer.StochasticBalancerTestBase.loadBalancer; +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.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MasterTests.class, MediumTests.class }) +public class TestPrefetchCacheCostLoadBalancerFunction extends StochasticBalancerTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPrefetchCacheCostLoadBalancerFunction.class); + + private static final String REGION_KEY = "testRegion"; + + // Mapping of prefetch test -> expected prefetch + private 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 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[] { 2, 0, 0, 100 }, // region 0 is hosted and prefetched only on server 2 + new int[] { 0, 100, 0, 0 }, // region 1 is hosted and prefetched only on server 0 + new int[] { 0, 100, 0, 0 }, // region 2 is hosted and prefetched only on server 0 + new int[] { 1, 0, 100, 0 }, // region 3 is hosted and prefetched only on server 1 + }, + + // 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[] { 2, 100, 0, 0 }, // Region 1 is currently hosted but not prefetched on server 2, + // but was fully prefetched on server 0 previously + new int[] { 1, 0, 0, 100 }, // Region 2 is currently hosted but not prefetched on server 1, + // but was fully prefetched on server 2 previously + new int[] { 2, 0, 100, 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[] { 1, 50, 50, 0 }, // Region 0 is hosted and partially prefetched on server 1, but + // was previously hosted and partially prefetched on server 0 + new int[] { 2, 0, 50, 50 }, // Region 1 is hosted and partially prefetched on server 2, but + // was previously hosted and partially prefetched on server 1 + new int[] { 0, 50, 0, 50 }, // Region 2 is hosted and partially prefetched on server 0, but + // was previously hosted and partially prefetched on server 2 + new int[] { 0, 50, 50, 0 }, // Region 3 is hosted and partially prefetched on server 0, 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[] { 2, 70, 0, 30 }, // Region 1 is hosted and prefetched 30% on server 2, but was + // previously hosted and prefetched 70% on server 0 + new int[] { 1, 70, 30, 0 }, // Region 2 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 3 is hosted and prefetched 30% on server 1, but was + // previously hosted and prefetched 70% on server 3 + }, + + // 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[] { 2, 0, 20, 80 }, // Region 0 is hosted and 80% prefetched on server 2, but was + // previously hosted and 20% prefetched on server 1 + new int[] { 2, 20, 0, 80 }, // Region 1 is hosted and 80% prefetched on server 2, but was + // previously hosted and 20% prefetched on server 0 + 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[] { 0, 80, 20, 0 }, // Region 3 is hosted and 80% prefetched on server 0, 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[] { 1, 0, 34, 58 }, // Region 0 is hosted and partially prefetched on server 1, + // but was previously hosted and partially prefetched on server 2 + // current prefetch < historical prefetch + new int[] { 2, 78, 0, 100 }, // Region 1 is hosted and fully prefetched on server 2, + // 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[] { 0, 96, 0, 0 }, // Region 3 is hosted and partially prefetched on server 0 + // No historical prefetch + }, + }; + + @Test + public void testVerifyPrefetchCostFunctionEnabled() { + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch.persistence"); + + StochasticLoadBalancer lb = new StochasticLoadBalancer(); + lb.loadConf(conf); + + assertTrue(Arrays.asList(lb.getCostFunctionNames()) + .contains(PrefetchCacheCostFunction.class.getSimpleName())); + } + + @Test + public void testVerifyPrefetchCostFunctionDisabled() { + assertFalse(Arrays.asList(loadBalancer.getCostFunctionNames()) + .contains(PrefetchCacheCostFunction.class.getSimpleName())); + } + + @Test + public void testPrefetchCost() throws Exception { + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch.persistence"); + CostFunction costFunction = new PrefetchCacheCostFunction(conf); + + for (int test = 0; test < clusterRegionPrefetchMocks.length; test++) { + int[][] clusterRegionLocations = clusterRegionPrefetchMocks[test]; + TestPrefetchCacheCostLoadBalancerFunction.MockClusterForPrefetch + cluster = new TestPrefetchCacheCostLoadBalancerFunction.MockClusterForPrefetch(clusterRegionLocations); + costFunction.prepare(cluster); + double cost = costFunction.cost(); + assertEquals(expectedPrefetch[test], cost, 0.01); + } + } + + private class MockClusterForPrefetch extends BalancerClusterState { + private int[][] regionServerPrefetch = null; // [region][server] = prefetch percent + + public MockClusterForPrefetch(int[][] regionsArray) { + // regions[0] is an array where index = serverIndex and value = number of regions + super(mockClusterServers(regionsArray[0], 1), null, null, null, null); + regionServerPrefetch = new int[regionsArray.length - 1][]; + Map> historicalPrefetchRatio = + new HashMap>(); + for (int i = 1; i < regionsArray.length; i++) { + int regionIndex = i - 1; + regionServerPrefetch[regionIndex] = new int[regionsArray[i].length - 1]; + regionIndexToServerIndex[regionIndex] = regionsArray[i][0]; + for (int j = 1; j < regionsArray[i].length; j++) { + int serverIndex = j - 1; + regionServerPrefetch[regionIndex][serverIndex] = regionsArray[i][j]; + if (regionsArray[i][j] > 0 && serverIndex != regionsArray[i][0]) { + // This is the historical prefetch value + Map historicalPrefetch = new HashMap<>(); + historicalPrefetch.put(servers[serverIndex].getServerName(), (float)regionsArray[i][j]); + historicalPrefetchRatio.put(regions[regionIndex].getRegionNameAsString(), + historicalPrefetch); + } + } + } + historicalRegionServerPrefetchRatio = historicalPrefetchRatio; + } + + @Override public int getRegionSizeMB(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[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 regionNameAsString = regions[region].getRegionNameAsString(); + String serverNameAsString = servers[regionServerIndex].getServerName(); + if (historicalRegionServerPrefetchRatio != null && + historicalRegionServerPrefetchRatio.containsKey(regionNameAsString)) { + Map serverPrefetchRatio = historicalRegionServerPrefetchRatio.get(regionNameAsString); + if (serverPrefetchRatio.containsKey(serverNameAsString)) { + prefetchRatio = serverPrefetchRatio.get(serverNameAsString); + // The old prefetch cache ratio has been accounted for and hence, clear up this information + // as it is not needed anymore + historicalRegionServerPrefetchRatio.remove(regionNameAsString, serverPrefetchRatio); + } + } + return prefetchRatio; + } + } +} 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 748e4814fd3b..0848f6c97784 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 @@ -66,47 +66,6 @@ public class TestStochasticLoadBalancer extends StochasticBalancerTestBase { private static final String REGION_KEY = "testRegion"; - // Mapping of prefetch test -> expected prefetch - private float[] expectedPrefetch = { 0.0f, 1.0f, 0.5f, 0.75f, 0.0f }; - - /** - * 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 int[][][] clusterRegionPrefetchMocks = new int[][][] { - // Test 1: each region is entirely on server that hosts it - new int[][] { new int[] { 2, 1, 1 }, new int[] { 2, 0, 0, 100 }, // region 0 is hosted and - // entirely prefethced on - // server 2 - new int[] { 0, 100, 0, 0 }, // region 1 is hosted and entirely prefetched on server 0 - new int[] { 0, 100, 0, 0 }, // region 2 is hosted and entirely prefetched on server 0 - new int[] { 1, 0, 100, 0 }, // region 3 is hosted and entirely prefetched on server 1 - }, - - // Test 2: each region is 0% local on the server that hosts it - new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 0, 0, 100 }, // region 0 is hosted and - // entirely prefetched on - // server 2 - new int[] { 1, 100, 0, 0 }, // region 1 is hosted and prefetched entirely on server 0 - new int[] { 1, 100, 0, 0 }, // region 2 is hosted and prefetched entirely on server 0 - new int[] { 2, 0, 100, 0 }, // region 3 is hosted and prefetched entirely on server 1 - }, - - // Test 3: each region is 25% prefetched on the server that hosts it while 50% prefetched - // on some other server - new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 25, 0, 50 }, new int[] { 1, 50, 25, 0 }, - new int[] { 1, 50, 25, 0 }, new int[] { 2, 0, 50, 25 }, }, - - // Test 4: each region is 25% prefetched on the server that hosts it and 100% prefetched - // on some other server - new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 25, 0, 100 }, new int[] { 1, 100, 25, 0 }, - new int[] { 1, 100, 25, 0 }, new int[] { 2, 0, 100, 25 }, }, - - // Test 5: each region is 75% prefetched on all the servers - new int[][] { new int[] { 1, 2, 1 }, new int[] { 0, 75, 75, 75 }, new int[] { 1, 75, 75, 75 }, - new int[] { 1, 75, 75, 75 }, new int[] { 2, 75, 75, 75 }, }, }; - // Mapping of locality test -> expected locality private float[] expectedLocalities = { 1.0f, 0.0f, 0.50f, 0.25f, 1.0f }; private static Configuration storedConfiguration; @@ -359,6 +318,7 @@ public void testUpdateStochasticCostsIfBalanceNotRan() { @Test public void testNeedBalance() { conf.setFloat("hbase.master.balancer.stochastic.minCostNeedBalance", 1.0f); + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch_persistence"); conf.setFloat(HConstants.LOAD_BALANCER_SLOP_KEY, -1f); conf.setBoolean("hbase.master.balancer.stochastic.runMaxSteps", false); conf.setLong("hbase.master.balancer.stochastic.maxSteps", 5000L); @@ -645,6 +605,8 @@ public void testAdditionalCostFunction() { @Test public void testDefaultCostFunctionList() { + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch_persistence"); + loadBalancer.loadConf(conf); List expected = Arrays.asList(RegionCountSkewCostFunction.class.getSimpleName(), PrimaryRegionCountSkewCostFunction.class.getSimpleName(), MoveCostFunction.class.getSimpleName(), RackLocalityCostFunction.class.getSimpleName(), @@ -653,7 +615,8 @@ public void testDefaultCostFunctionList() { RegionReplicaRackCostFunction.class.getSimpleName(), ReadRequestCostFunction.class.getSimpleName(), CPRequestCostFunction.class.getSimpleName(), WriteRequestCostFunction.class.getSimpleName(), - MemStoreSizeCostFunction.class.getSimpleName(), StoreFileCostFunction.class.getSimpleName()); + MemStoreSizeCostFunction.class.getSimpleName(), StoreFileCostFunction.class.getSimpleName(), + PrefetchCacheCostFunction.class.getSimpleName()); List actual = Arrays.asList(loadBalancer.getCostFunctionNames()); assertTrue("ExpectedCostFunctions: " + expected + " ActualCostFunctions: " + actual, @@ -665,72 +628,6 @@ private boolean needsBalanceIdleRegion(int[] cluster) { && Arrays.stream(cluster).anyMatch(x -> x < 1); } - @Test - public void testVerifyPrefetchCostFunctionEnabled() { - conf.set("hbase.prefetch.file-list.path", "/tmp/prefetch.persistence"); - - StochasticLoadBalancer lb = new StochasticLoadBalancer(); - lb.loadConf(conf); - - assertTrue(Arrays.asList(lb.getCostFunctionNames()) - .contains(PrefetchCacheCostFunction.class.getSimpleName())); - } - - @Test - public void testVerifyPrefetchCostFunctionNotEnabled() { - assertFalse(Arrays.asList(loadBalancer.getCostFunctionNames()) - .contains(PrefetchCacheCostFunction.class.getSimpleName())); - } - - @Test - public void testPrefetchCost() throws Exception { - conf.set("hbase.prefetch.file-list.path", "/tmp/prefetch.persistence"); - CostFunction costFunction = new PrefetchCacheCostFunction(conf); - - for (int test = 0; test < clusterRegionPrefetchMocks.length; test++) { - int[][] clusterRegionLocations = clusterRegionPrefetchMocks[test]; - MockClusterForPrefetch cluster = new MockClusterForPrefetch(clusterRegionLocations); - costFunction.prepare(cluster); - double cost = costFunction.cost(); - assertEquals(expectedPrefetch[test], cost, 0.01); - } - } - - private class MockClusterForPrefetch extends BalancerClusterState { - private int[][] regionServerPrefetch = null; // [region][server] = prefetch percent - - public MockClusterForPrefetch(int[][] regions) { - // regions[0] is an array where index = serverIndex and value = number of regions - super(mockClusterServers(regions[0], 1), null, null, null); - regionServerPrefetch = new int[regions.length - 1][]; - for (int i = 1; i < regions.length; i++) { - int regionIndex = i - 1; - regionServerPrefetch[regionIndex] = new int[regions[i].length - 1]; - regionIndexToServerIndex[regionIndex] = regions[i][0]; - for (int j = 1; j < regions[i].length; j++) { - int serverIndex = j - 1; - regionServerPrefetch[regionIndex][serverIndex] = regions[i][j]; - } - } - } - - @Override - public float getOrComputeWeightedPrefetchRatio(int region, int server) { - return getRegionSizeMB(region) * regionServerPrefetch[region][server] / 100.0f; - } - - @Override - public int getRegionSizeMB(int region) { - return 1; - } - - @Override - protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) { - return regionServerPrefetch[region][regionServerIndex] / 100.0f; - } - } - - // This mock allows us to test the LocalityCostFunction private class MockCluster extends BalancerClusterState { private int[][] localities = null; // [region][server] = percent of blocks diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java index 2a7b8afccc38..16c8ef625883 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerBalanceCluster.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -51,6 +52,7 @@ public class TestStochasticLoadBalancerBalanceCluster extends StochasticBalancer */ @Test public void testBalanceCluster() throws Exception { + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch_persistence"); loadBalancer.onConfigurationChange(conf); for (int[] mockCluster : clusterStateMocks) { Map> servers = mockClusterServers(mockCluster); diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java index 620d610288f7..e02728a7e32c 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerLargeCluster.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.balancer; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.ClassRule; @@ -39,6 +40,7 @@ public void testLargeCluster() { int numTables = 100; int replication = 1; conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 6 * 60 * 1000); + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch_persistence"); loadBalancer.onConfigurationChange(conf); testWithClusterWithIteration(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java index a58b8e162968..aac7247afd24 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplicaHighReplication.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.balancer; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.ClassRule; @@ -36,6 +37,7 @@ public class TestStochasticLoadBalancerRegionReplicaHighReplication public void testRegionReplicasOnMidClusterHighReplication() { conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 4000000L); conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch_persistence"); loadBalancer.onConfigurationChange(conf); int numNodes = 40; int numRegions = 6 * numNodes; diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java index 831c9e932727..925de0e06240 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerSmallCluster.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.balancer; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.ClassRule; @@ -38,6 +39,7 @@ public void testSmallCluster() { int numRegionsPerServer = 40; // all servers except one int replication = 1; int numTables = 10; + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch_persistence"); testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); } @@ -48,6 +50,7 @@ public void testSmallCluster2() { int numRegionsPerServer = 40; // all servers except one int replication = 1; int numTables = 10; + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch_persistence"); testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true); } @@ -58,6 +61,7 @@ public void testSmallCluster3() { int numRegionsPerServer = 1; // all servers except one int replication = 1; int numTables = 10; + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch_persistence"); // fails because of max moves testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, false, false); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java index f07650147b4f..0ed25f123005 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java @@ -141,4 +141,7 @@ default String getNameAsString() { /** Returns the ratio of files already in cache */ float getPrefetchCacheRatio(); + + /** Returns the name of the server hosting this region */ + String getServerName(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java index d04534cf0662..102cc1d56d7a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java @@ -80,7 +80,8 @@ public static RegionMetrics toRegionMetrics(ClusterStatusProtos.RegionLoad regio ClusterStatusProtos.StoreSequenceId::getSequenceId))) .setUncompressedStoreFileSize( new Size(regionLoadPB.getStoreUncompressedSizeMB(), Size.Unit.MEGABYTE)) - .setPrefetchCacheRatio(regionLoadPB.getPrefetchCacheRatio()).build(); + .setPrefetchCacheRatio(regionLoadPB.getPrefetchCacheRatio()) + .setServerName(regionLoadPB.getServerName()).build(); } private static List @@ -120,7 +121,8 @@ public static ClusterStatusProtos.RegionLoad toRegionLoad(RegionMetrics regionMe .addAllStoreCompleteSequenceId(toStoreSequenceId(regionMetrics.getStoreSequenceId())) .setStoreUncompressedSizeMB( (int) regionMetrics.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE)) - .setPrefetchCacheRatio(regionMetrics.getPrefetchCacheRatio()).build(); + .setPrefetchCacheRatio(regionMetrics.getPrefetchCacheRatio()) + .setServerName(regionMetrics.getServerName()).build(); } public static RegionMetricsBuilder newBuilder(byte[] name) { @@ -155,6 +157,7 @@ public static RegionMetricsBuilder newBuilder(byte[] name) { private long blocksTotalWeight; private CompactionState compactionState; private float prefetchCacheRatio; + private String serverName; private RegionMetricsBuilder(byte[] name) { this.name = name; @@ -295,6 +298,11 @@ public RegionMetricsBuilder setPrefetchCacheRatio(float prefetchCacheRatio) { return this; } + public RegionMetricsBuilder setServerName(String serverName) { + this.serverName = serverName; + return this; + } + public RegionMetrics build() { return new RegionMetricsImpl(name, storeCount, storeFileCount, storeRefCount, maxCompactedStoreFileRefCount, compactingCellCount, compactedCellCount, storeFileSize, @@ -302,7 +310,7 @@ public RegionMetrics build() { uncompressedStoreFileSize, writeRequestCount, readRequestCount, cpRequestCount, filteredReadRequestCount, completedSequenceId, storeSequenceIds, dataLocality, lastMajorCompactionTimestamp, dataLocalityForSsd, blocksLocalWeight, blocksLocalWithSsdWeight, - blocksTotalWeight, compactionState, prefetchCacheRatio); + blocksTotalWeight, compactionState, prefetchCacheRatio, serverName); } private static class RegionMetricsImpl implements RegionMetrics { @@ -334,6 +342,7 @@ private static class RegionMetricsImpl implements RegionMetrics { private final long blocksTotalWeight; private final CompactionState compactionState; private final float prefetchCacheRatio; + private final String serverName; RegionMetricsImpl(byte[] name, int storeCount, int storeFileCount, int storeRefCount, int maxCompactedStoreFileRefCount, final long compactingCellCount, long compactedCellCount, @@ -343,7 +352,7 @@ private static class RegionMetricsImpl implements RegionMetrics { long filteredReadRequestCount, long completedSequenceId, Map storeSequenceIds, float dataLocality, long lastMajorCompactionTimestamp, float dataLocalityForSsd, long blocksLocalWeight, long blocksLocalWithSsdWeight, long blocksTotalWeight, - CompactionState compactionState, float prefetchCacheRatio) { + CompactionState compactionState, float prefetchCacheRatio, String serverName) { this.name = Preconditions.checkNotNull(name); this.storeCount = storeCount; this.storeFileCount = storeFileCount; @@ -372,6 +381,7 @@ private static class RegionMetricsImpl implements RegionMetrics { this.blocksTotalWeight = blocksTotalWeight; this.compactionState = compactionState; this.prefetchCacheRatio = prefetchCacheRatio; + this.serverName = serverName; } @Override @@ -514,6 +524,11 @@ public float getPrefetchCacheRatio() { return prefetchCacheRatio; } + @Override + public String getServerName() { + return serverName; + } + @Override public String toString() { StringBuilder sb = @@ -555,6 +570,7 @@ public String toString() { Strings.appendKeyValue(sb, "blocksTotalWeight", blocksTotalWeight); Strings.appendKeyValue(sb, "compactionState", compactionState); Strings.appendKeyValue(sb, "prefetchCacheRatio", this.getPrefetchCacheRatio()); + Strings.appendKeyValue(sb, "serverName", this.getServerName()); return sb.toString(); } } 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 dac7de6eb5b4..b846f304c13e 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 @@ -1316,6 +1316,11 @@ public enum OperationStatusCode { // sits in hbase-server, there were no other go! Can we move the cache implementation to // hbase-common? + /** + * Prefetch persistence path key + */ + public static final String PREFETCH_PERSISTENCE_PATH_KEY = "hbase.prefetch.file.list.path"; + /** * Current ioengine options in include: heap, offheap and file:PATH (where PATH is the path to the * file that will host the file-based cache. See BucketCache#getIOEngineFromName() for list of diff --git a/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto b/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto index 938f41efc30a..b994308e966e 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto @@ -180,6 +180,9 @@ message RegionLoad { /** The prefetch cache ratio for region */ optional float prefetch_cache_ratio = 28; + + /** The server hosting this region */ + optional string server_name = 29; } message UserLoad { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java index 27f75e4eee62..88e9bf2105dd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java @@ -93,8 +93,6 @@ public class CacheConfig { public static final String DROP_BEHIND_CACHE_COMPACTION_KEY = "hbase.hfile.drop.behind.compaction"; - public static final String PREFETCH_PERSISTENCE_PATH_KEY = "hbase.prefetch.file-list.path"; - // Defaults public static final boolean DEFAULT_CACHE_DATA_ON_READ = true; public static final boolean DEFAULT_CACHE_DATA_ON_WRITE = false; 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 2e2ae4865653..088d6ab8cd81 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 @@ -17,8 +17,6 @@ */ package org.apache.hadoop.hbase.io.hfile.bucket; -import static org.apache.hadoop.hbase.io.hfile.CacheConfig.PREFETCH_PERSISTENCE_PATH_KEY; - import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; @@ -51,6 +49,7 @@ import java.util.function.Function; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.io.ByteBuffAllocator; @@ -288,7 +287,7 @@ public BucketCache(String ioEngineName, long capacity, int blockSize, int[] buck this.singleFactor = conf.getFloat(SINGLE_FACTOR_CONFIG_NAME, DEFAULT_SINGLE_FACTOR); this.multiFactor = conf.getFloat(MULTI_FACTOR_CONFIG_NAME, DEFAULT_MULTI_FACTOR); this.memoryFactor = conf.getFloat(MEMORY_FACTOR_CONFIG_NAME, DEFAULT_MEMORY_FACTOR); - this.prefetchedFileListPath = conf.get(PREFETCH_PERSISTENCE_PATH_KEY); + this.prefetchedFileListPath = conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY); sanityCheckConfigs(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 626fb3a5cae7..c498c2be771a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1571,7 +1571,8 @@ RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, .setBlocksLocalWithSsdWeight(blocksLocalWithSsdWeight).setBlocksTotalWeight(blocksTotalWeight) .setCompactionState(ProtobufUtil.createCompactionStateForRegionLoad(r.getCompactionState())) .setLastMajorCompactionTs(r.getOldestHfileTs(true)) - .setPrefetchCacheRatio(ratioOfFilesAlreadyCached); + .setPrefetchCacheRatio(ratioOfFilesAlreadyCached) + .setServerName(serverName.getServerName()); r.setCompleteSequenceId(regionLoadBldr); return regionLoadBldr.build(); } 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 68f4371bb4ff..d9a602a1a715 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 @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; @@ -75,7 +76,7 @@ public void setup() throws Exception { conf.set(BUCKET_CACHE_IOENGINE_KEY, "file:" + testDir + "/bucket.cache"); conf.setInt("hbase.bucketcache.size", 400); conf.set("hbase.bucketcache.persistent.path", testDir + "/bucket.persistence"); - conf.set(CacheConfig.PREFETCH_PERSISTENCE_PATH_KEY, testDir + "/prefetch.persistence"); + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, testDir + "/prefetch.persistence"); zkCluster = TEST_UTIL.startMiniZKCluster(); cluster = TEST_UTIL.startMiniHBaseCluster(option); assertEquals(2, cluster.getRegionServerThreads().size()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestPrefetchPersistence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestPrefetchPersistence.java index 6a5c9dcf2db3..d7d1d95e3e45 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestPrefetchPersistence.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestPrefetchPersistence.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; @@ -107,7 +108,7 @@ public void setup() throws IOException { testDir = TEST_UTIL.getDataTestDir(); TEST_UTIL.getTestFileSystem().mkdirs(testDir); prefetchPersistencePath = testDir + "/prefetch.persistence"; - conf.set(CacheConfig.PREFETCH_PERSISTENCE_PATH_KEY, prefetchPersistencePath); + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, prefetchPersistencePath); fs = HFileSystem.get(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java index 28f92ac319dc..44deff55e92a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java @@ -546,6 +546,9 @@ public CompactionState getCompactionState() { public float getPrefetchCacheRatio() { return 0.0f; } + + @Override + public String getServerName() { return null; } }; return regionMetrics; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerRejection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerRejection.java index c5d8223479e8..c1c2c62cc48c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerRejection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerRejection.java @@ -24,6 +24,7 @@ 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.TableName; import org.apache.hadoop.hbase.client.LogEntry; @@ -75,6 +76,7 @@ public void testBalancerRejections() throws Exception { try { // enabled balancer rejection recording conf.setBoolean(BaseLoadBalancer.BALANCER_REJECTION_BUFFER_ENABLED, true); + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch_persistence"); conf.set(StochasticLoadBalancer.COST_FUNCTIONS_COST_FUNCTIONS_KEY, MockCostFunction.class.getName()); MasterServices services = mock(MasterServices.class); From a53a3fcae630f518cf55e1dbed5ff627a235fecb Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Wed, 18 Jan 2023 16:50:17 +0530 Subject: [PATCH 04/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions Fixed spotless errors --- .../master/balancer/BalancerClusterState.java | 39 +++++---- .../master/balancer/BalancerRegionLoad.java | 4 +- .../master/balancer/BaseLoadBalancer.java | 3 +- .../PrefetchBasedCandidateGenerator.java | 7 +- .../balancer/PrefetchCacheCostFunction.java | 5 +- .../balancer/StochasticLoadBalancer.java | 10 +-- ...PrefetchCacheCostLoadBalancerFunction.java | 87 ++++++++++++------- .../hbase/regionserver/HRegionServer.java | 3 +- .../master/TestRegionsRecoveryChore.java | 4 +- 9 files changed, 98 insertions(+), 64 deletions(-) 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 6787c584d950..9e732fa652a9 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 @@ -586,8 +586,8 @@ protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) // The region is currently hosted on this region server. Now, get the prefetch cache ratio // for this region on this region server - prefetchRatio = regionLoadList == null ? 0.0f : regionLoadList.getLast() - .getPrefetchCacheRatio(); + prefetchRatio = + regionLoadList == null ? 0.0f : regionLoadList.getLast().getPrefetchCacheRatio(); return prefetchRatio; } @@ -598,8 +598,10 @@ protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) // may change when a server is marked as dead or a new server is added. String regionNameAsString = regions[region].getRegionNameAsString(); String serverNameAsString = servers[regionServerIndex].getServerName(); - if (historicalRegionServerPrefetchRatio != null && - historicalRegionServerPrefetchRatio.containsKey(regionNameAsString)) { + if ( + historicalRegionServerPrefetchRatio != null + && historicalRegionServerPrefetchRatio.containsKey(regionNameAsString) + ) { Map serverPrefetchRatio = historicalRegionServerPrefetchRatio.get(regionNameAsString); if (serverPrefetchRatio.containsKey(serverNameAsString)) { @@ -633,32 +635,37 @@ private void computeRegionServerPrefetchRatio() { // 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 the same prefetch ratio, the the server currently hostring the - // region should retain the region - serverWithBestPrefetchRatio = server; - } + } else + if (prefetchRatio == bestPrefetchRatio && server == regionIndexToServerIndex[region]) { + // If two servers have the same prefetch ratio, the the server currently hostring the + // region should retain the region + serverWithBestPrefetchRatio = server; + } } regionServerIndexWithBestPrefetchRatio[region] = serverWithBestPrefetchRatio; } } private float getOrComputeRegionPrefetchRatio(int region, int server) { - if (regionServerIndexWithBestPrefetchRatio == null || - regionIndexServerIndexPrefetchRatio.isEmpty()) { + if ( + regionServerIndexWithBestPrefetchRatio == null + || regionIndexServerIndexPrefetchRatio.isEmpty() + ) { computeRegionServerPrefetchRatio(); } Map tempMap = new HashMap<>(); tempMap.put(region, server); - return regionIndexServerIndexPrefetchRatio.containsKey(tempMap) ? - regionIndexServerIndexPrefetchRatio.get(tempMap) : 0.0f; + return regionIndexServerIndexPrefetchRatio.containsKey(tempMap) + ? regionIndexServerIndexPrefetchRatio.get(tempMap) + : 0.0f; } public int[] getOrComputeServerWithBestPrefetchRatio() { - if (regionIndexServerIndexPrefetchRatio.isEmpty() || - regionServerIndexWithBestPrefetchRatio == null) { + if ( + regionIndexServerIndexPrefetchRatio.isEmpty() + || regionServerIndexWithBestPrefetchRatio == null + ) { computeRegionServerPrefetchRatio(); } return regionServerIndexWithBestPrefetchRatio; 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 20d2e764221f..c925d37d3a19 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,5 +71,7 @@ public float getPrefetchCacheRatio() { return prefetchCacheRatio; } - public String getServerName() { return serverName; } + public String getServerName() { + return serverName; + } } 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 92968e558255..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, null); + 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/PrefetchBasedCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java index 6e75fcce52ef..3244973fbcad 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.master.balancer; -import java.util.Optional; import java.util.concurrent.ThreadLocalRandom; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -48,8 +47,10 @@ BalanceAction generate(BalancerClusterState cluster) { cluster.getOrComputeWeightedPrefetchRatio(region, toServerIndex); // If the prefetch ratio on the target server is significantly higher, move the region. - if (currentPrefetchRatio > 0 && - (toServerPrefetchRatio / currentPrefetchRatio) > PREFETCH_RATIO_DIFF_FACTOR) { + if ( + currentPrefetchRatio > 0 + && (toServerPrefetchRatio / currentPrefetchRatio) > PREFETCH_RATIO_DIFF_FACTOR + ) { return getAction(currentServerIndex, region, toServerIndex, -1); } } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java index ee66c6971e39..b13133bfd0e6 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -39,8 +39,9 @@ public class PrefetchCacheCostFunction extends CostFunction { prefetchedFileListPath = conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY); // Disable the prefetch cache cost function if the prefetched file list persistence is not // enabled - this.setMultiplier(prefetchedFileListPath == null ? 0.0f : - conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_COST)); + this.setMultiplier(prefetchedFileListPath == null + ? 0.0f + : conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_COST)); prefetchRatio = 0.0f; bestPrefetchRatio = 0.0f; } 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 0d568ec0af6f..f0c1351b16e1 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 @@ -301,9 +301,8 @@ private void updateBalancerTableLoadInfo(TableName tableName, if ((this.localityCost != null) || (this.rackLocalityCost != null)) { finder = this.regionFinder; } - BalancerClusterState cluster = - new BalancerClusterState(loadOfOneTable, loads, finder, rackManager, - historicRegionServerPrefetchRatio); + BalancerClusterState cluster = new BalancerClusterState(loadOfOneTable, loads, finder, + rackManager, historicRegionServerPrefetchRatio); initCosts(cluster); curOverallCost = computeCost(cluster, Double.MAX_VALUE); @@ -474,9 +473,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, - historicRegionServerPrefetchRatio); + BalancerClusterState cluster = new BalancerClusterState(loadOfOneTable, loads, finder, + rackManager, historicRegionServerPrefetchRatio); long startTime = EnvironmentEdgeManager.currentTime(); diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java index 58d8bc76dc83..ceb5d5c769a1 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -21,6 +21,7 @@ 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; @@ -82,9 +83,12 @@ public class TestPrefetchCacheCostLoadBalancerFunction extends StochasticBalance // 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[][] { 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 @@ -96,22 +100,27 @@ public class TestPrefetchCacheCostLoadBalancerFunction extends StochasticBalance // 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[][] { 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[] { 2, 100, 0, 0 }, // Region 1 is currently hosted but not prefetched on server 2, // but was fully prefetched on server 0 previously new int[] { 1, 0, 0, 100 }, // Region 2 is currently hosted but not prefetched on server 1, // but was fully prefetched on server 2 previously - new int[] { 2, 0, 100, 0 }, // Region 3 is currently hosted but not prefetched on server 2, - // but was fully prefetched on server 1 previously + new int[] { 2, 0, 100, 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[] { 1, 50, 50, 0 }, // Region 0 is hosted and partially prefetched on server 1, but - // was previously hosted and partially prefetched on server 0 + new int[][] { new int[] { 2, 1, 1 }, new int[] { 1, 50, 50, 0 }, // Region 0 is hosted and + // partially prefetched on + // server 1, but + // was previously hosted and + // partially prefetched on + // server 0 new int[] { 2, 0, 50, 50 }, // Region 1 is hosted and partially prefetched on server 2, but // was previously hosted and partially prefetched on server 1 new int[] { 0, 50, 0, 50 }, // Region 2 is hosted and partially prefetched on server 0, but @@ -122,9 +131,11 @@ public class TestPrefetchCacheCostLoadBalancerFunction extends StochasticBalance // 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[][] { 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[] { 2, 70, 0, 30 }, // Region 1 is hosted and prefetched 30% on server 2, but was // previously hosted and prefetched 70% on server 0 new int[] { 1, 70, 30, 0 }, // Region 2 is hosted and prefetched 30% on server 1, but was @@ -135,9 +146,11 @@ public class TestPrefetchCacheCostLoadBalancerFunction extends StochasticBalance // 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[] { 2, 0, 20, 80 }, // Region 0 is hosted and 80% prefetched on server 2, but was - // previously hosted and 20% prefetched on server 1 + new int[][] { new int[] { 2, 1, 1 }, new int[] { 2, 0, 20, 80 }, // Region 0 is hosted and 80% + // prefetched on server 2, but + // was + // previously hosted and 20% + // prefetched on server 1 new int[] { 2, 20, 0, 80 }, // Region 1 is hosted and 80% prefetched on server 2, but was // previously hosted and 20% prefetched on server 0 new int[] { 1, 20, 80, 0 }, // Region 2 is hosted and 80% prefetched on server 1, but was @@ -148,10 +161,14 @@ public class TestPrefetchCacheCostLoadBalancerFunction extends StochasticBalance // 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[] { 1, 0, 34, 58 }, // Region 0 is hosted and partially prefetched on server 1, - // but was previously hosted and partially prefetched on server 2 - // current prefetch < historical prefetch + new int[][] { new int[] { 1, 2, 1 }, new int[] { 1, 0, 34, 58 }, // Region 0 is hosted and + // partially prefetched on + // server 1, + // but was previously hosted + // and partially prefetched on + // server 2 + // current prefetch < + // historical prefetch new int[] { 2, 78, 0, 100 }, // Region 1 is hosted and fully prefetched on server 2, // but was previously hosted and partially prefetched on server 0 // current prefetch > historical prefetch @@ -160,8 +177,7 @@ public class TestPrefetchCacheCostLoadBalancerFunction extends StochasticBalance // current prefetch == historical prefetch new int[] { 0, 96, 0, 0 }, // Region 3 is hosted and partially prefetched on server 0 // No historical prefetch - }, - }; + }, }; @Test public void testVerifyPrefetchCostFunctionEnabled() { @@ -187,8 +203,9 @@ public void testPrefetchCost() throws Exception { for (int test = 0; test < clusterRegionPrefetchMocks.length; test++) { int[][] clusterRegionLocations = clusterRegionPrefetchMocks[test]; - TestPrefetchCacheCostLoadBalancerFunction.MockClusterForPrefetch - cluster = new TestPrefetchCacheCostLoadBalancerFunction.MockClusterForPrefetch(clusterRegionLocations); + TestPrefetchCacheCostLoadBalancerFunction.MockClusterForPrefetch cluster = + new TestPrefetchCacheCostLoadBalancerFunction.MockClusterForPrefetch( + clusterRegionLocations); costFunction.prepare(cluster); double cost = costFunction.cost(); assertEquals(expectedPrefetch[test], cost, 0.01); @@ -214,7 +231,8 @@ public MockClusterForPrefetch(int[][] regionsArray) { if (regionsArray[i][j] > 0 && serverIndex != regionsArray[i][0]) { // This is the historical prefetch value Map historicalPrefetch = new HashMap<>(); - historicalPrefetch.put(servers[serverIndex].getServerName(), (float)regionsArray[i][j]); + historicalPrefetch.put(servers[serverIndex].getServerName(), + (float) regionsArray[i][j]); historicalPrefetchRatio.put(regions[regionIndex].getRegionNameAsString(), historicalPrefetch); } @@ -223,7 +241,8 @@ public MockClusterForPrefetch(int[][] regionsArray) { historicalRegionServerPrefetchRatio = historicalPrefetchRatio; } - @Override public int getRegionSizeMB(int region) { + @Override + public int getRegionSizeMB(int region) { return 1; } @@ -242,12 +261,16 @@ protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) // ids may change when a server is marked as dead or a new server is added. String regionNameAsString = regions[region].getRegionNameAsString(); String serverNameAsString = servers[regionServerIndex].getServerName(); - if (historicalRegionServerPrefetchRatio != null && - historicalRegionServerPrefetchRatio.containsKey(regionNameAsString)) { - Map serverPrefetchRatio = historicalRegionServerPrefetchRatio.get(regionNameAsString); + if ( + historicalRegionServerPrefetchRatio != null + && historicalRegionServerPrefetchRatio.containsKey(regionNameAsString) + ) { + Map serverPrefetchRatio = + historicalRegionServerPrefetchRatio.get(regionNameAsString); if (serverPrefetchRatio.containsKey(serverNameAsString)) { prefetchRatio = serverPrefetchRatio.get(serverNameAsString); - // The old prefetch cache ratio has been accounted for and hence, clear up this information + // The old prefetch cache ratio has been accounted for and hence, clear up this + // information // as it is not needed anymore historicalRegionServerPrefetchRatio.remove(regionNameAsString, serverPrefetchRatio); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index b484231b5d55..807080b4cfdd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1608,8 +1608,7 @@ RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, .setBlocksLocalWithSsdWeight(blocksLocalWithSsdWeight).setBlocksTotalWeight(blocksTotalWeight) .setCompactionState(ProtobufUtil.createCompactionStateForRegionLoad(r.getCompactionState())) .setLastMajorCompactionTs(r.getOldestHfileTs(true)) - .setPrefetchCacheRatio(ratioOfFilesAlreadyCached) - .setServerName(serverName.getServerName()); + .setPrefetchCacheRatio(ratioOfFilesAlreadyCached).setServerName(serverName.getServerName()); r.setCompleteSequenceId(regionLoadBldr); return regionLoadBldr.build(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java index 44deff55e92a..9f8b86c6c0e8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java @@ -548,7 +548,9 @@ public float getPrefetchCacheRatio() { } @Override - public String getServerName() { return null; } + public String getServerName() { + return null; + } }; return regionMetrics; } From 826db4667ec41d8def5033450214acdf9414004f Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Thu, 2 Mar 2023 17:52:14 +0530 Subject: [PATCH 05/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions Addressed review comments --- .../master/balancer/BalancerClusterState.java | 49 +++++++++++++------ .../balancer/PrefetchCacheCostFunction.java | 12 ++++- .../balancer/StochasticLoadBalancer.java | 1 - 3 files changed, 45 insertions(+), 17 deletions(-) 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 9e732fa652a9..422a4a0f3741 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; @@ -116,7 +117,7 @@ class BalancerClusterState { private int[][] regionsToMostLocalEntities; // Maps region -> serverIndex -> prefetch ratio of a region on a server - private Map, Float> regionIndexServerIndexPrefetchRatio; + private Map, Float> regionIndexServerIndexPrefetchRatio; // Maps region -> serverIndex with best prefect ratio private int[] regionServerIndexWithBestPrefetchRatio; // Historical region server prefetch ratio @@ -569,10 +570,24 @@ enum LocalityType { RACK } + /** + * Returns prefetch ratio weighted to the size of the region in MB.* + * @param region Region ID + * @param server Server ID + * @return Weighted prefetch ratio of a region on a region server + */ public float getOrComputeWeightedPrefetchRatio(int region, int server) { return getRegionSizeMB(region) * getOrComputeRegionPrefetchRatio(region, server); } + /** + * Returns prefetch ratio of a region on a region server. + * This method also finds out the historical prefetch ratio of this region if it was ever hosted + * on this region server. + * @param region Region ID + * @param regionServerIndex Region Server ID + * @return Prefetch ratio of given region on the given region server + */ protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) { // Cost this server has from RegionLoad float prefetchRatio = 0.0f; @@ -614,6 +629,11 @@ protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) return prefetchRatio; } + /** + * Compute the prefetch ratios of all the regions on all the servers. This even includes the + * historical prefetch ratio of a region if it was historically hosted on some other region server + * before being moved to the currently hosting region server. + */ private void computeRegionServerPrefetchRatio() { regionIndexServerIndexPrefetchRatio = new HashMap<>(); regionServerIndexWithBestPrefetchRatio = new int[numRegions]; @@ -624,23 +644,25 @@ private void computeRegionServerPrefetchRatio() { for (int server = 0; server < numServers; server++) { float prefetchRatio = getRegionServerPrefetchRatio(region, server); if (prefetchRatio > 0.0f || server == regionIndexToServerIndex[region]) { - // A region with prefetch ratio of 0 on a server means nothing. Hence, just make a note - // of prefetch only if the prefetch ratio is greater than 0. - Map tempMap = new HashMap<>(); - tempMap.put(region, server); - regionIndexServerIndexPrefetchRatio.put(tempMap, prefetchRatio); + // Record the prefetch ratio of a region on a region server only if it is greater than 0 + // which means either the region is currently hosted or was previously hosted on the given + // region server. We don't need to record a prefetch ratio of 0 as it does not add any + // value to the balancer decisions. + 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 + } else { + // If the server currently hosting the region has equal prefetch ratio to a historical + // prefetch ratio, consider the current server to continue hosting the region if (prefetchRatio == bestPrefetchRatio && server == regionIndexToServerIndex[region]) { - // If two servers have the same prefetch ratio, the the server currently hostring the + // If two servers have the same prefetch ratio, the server currently hosting the // region should retain the region serverWithBestPrefetchRatio = server; } + } } regionServerIndexWithBestPrefetchRatio[region] = serverWithBestPrefetchRatio; } @@ -654,10 +676,9 @@ private float getOrComputeRegionPrefetchRatio(int region, int server) { computeRegionServerPrefetchRatio(); } - Map tempMap = new HashMap<>(); - tempMap.put(region, server); - return regionIndexServerIndexPrefetchRatio.containsKey(tempMap) - ? regionIndexServerIndexPrefetchRatio.get(tempMap) + Pair regionServerPair = new Pair<>(region, server); + return regionIndexServerIndexPrefetchRatio.containsKey(regionServerPair) + ? regionIndexServerIndexPrefetchRatio.get(regionServerPair) : 0.0f; } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java index b13133bfd0e6..dae3d23b2861 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -35,10 +35,18 @@ public class PrefetchCacheCostFunction extends CostFunction { private double prefetchRatio; private float bestPrefetchRatio; + /** + * Enables or disables the prefetch cache cost function depending on the parameter + * PREFETCH_PERSISTENCE_PATH_KEY. If set, this parameter enables the prefetched file list + * persistence. + * If this parameter is not set this means that the cache persistence is disabled which means + * that the prefetch ratios of regions on region servers cannot be calculated and hence the + * regions should be moved based on how much they have been prefetched on a region server. + * The prefetch cache cost function is disabled if the multiplier is set to 0. + * @param conf Cluster configuration + */ PrefetchCacheCostFunction(Configuration conf) { prefetchedFileListPath = conf.get(HConstants.PREFETCH_PERSISTENCE_PATH_KEY); - // Disable the prefetch cache cost function if the prefetched file list persistence is not - // enabled this.setMultiplier(prefetchedFileListPath == null ? 0.0f : conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_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 f0c1351b16e1..9f5a4b04a865 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 @@ -465,7 +465,6 @@ protected List balanceTable(TableName tableName, RegionHDFSBlockLocationFinder finder = null; if ( (this.localityCost != null) || (this.rackLocalityCost != null) - || (this.prefetchCacheCost != null) ) { finder = this.regionFinder; } From ab870c0c3e75d0a1e341b194f27efe7bf858be61 Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Thu, 2 Mar 2023 18:04:49 +0530 Subject: [PATCH 06/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions Added javadoc to StochasticLoadBalancer --- .../hadoop/hbase/master/balancer/StochasticLoadBalancer.java | 2 ++ 1 file changed, 2 insertions(+) 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 9f5a4b04a865..8d9d4c7370e7 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 @@ -59,6 +59,7 @@ *
  • Data Locality
  • *
  • Memstore Sizes
  • *
  • Storefile Sizes
  • + *
  • Prefetch
  • * *

    * Every cost function returns a number between 0 and 1 inclusive; where 0 is the lowest cost best @@ -72,6 +73,7 @@ *

  • hbase.master.balancer.stochastic.localityCost
  • *
  • hbase.master.balancer.stochastic.memstoreSizeCost
  • *
  • hbase.master.balancer.stochastic.storefileSizeCost
  • + *
  • hbase.master.balancer.stochastic.prefetchCacheCost
  • * *

    * You can also add custom Cost function by setting the the following configuration value: From a8dbcefcd39862b2896cc03cf691d8609b6e13f6 Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Thu, 2 Mar 2023 21:27:01 +0530 Subject: [PATCH 07/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions Fixed spotless failures --- .../hbase/master/balancer/BalancerClusterState.java | 7 +++---- .../hbase/master/balancer/PrefetchCacheCostFunction.java | 9 ++++----- .../hbase/master/balancer/StochasticLoadBalancer.java | 4 +--- 3 files changed, 8 insertions(+), 12 deletions(-) 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 422a4a0f3741..ee852a86fe51 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 @@ -581,10 +581,9 @@ public float getOrComputeWeightedPrefetchRatio(int region, int server) { } /** - * Returns prefetch ratio of a region on a region server. - * This method also finds out the historical prefetch ratio of this region if it was ever hosted - * on this region server. - * @param region Region ID + * Returns prefetch ratio of a region on a region server. This method also finds out the + * historical prefetch ratio of this region if it was ever hosted on this region server. + * @param region Region ID * @param regionServerIndex Region Server ID * @return Prefetch ratio of given region on the given region server */ diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java index dae3d23b2861..225b97b30e1a 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -38,11 +38,10 @@ public class PrefetchCacheCostFunction extends CostFunction { /** * Enables or disables the prefetch cache cost function depending on the parameter * PREFETCH_PERSISTENCE_PATH_KEY. If set, this parameter enables the prefetched file list - * persistence. - * If this parameter is not set this means that the cache persistence is disabled which means - * that the prefetch ratios of regions on region servers cannot be calculated and hence the - * regions should be moved based on how much they have been prefetched on a region server. - * The prefetch cache cost function is disabled if the multiplier is set to 0. + * persistence. If this parameter is not set this means that the cache persistence is disabled + * which means that the prefetch ratios of regions on region servers cannot be calculated and + * hence the regions should be moved based on how much they have been prefetched on a region + * server. The prefetch cache cost function is disabled if the multiplier is set to 0. * @param conf Cluster configuration */ PrefetchCacheCostFunction(Configuration conf) { 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 8d9d4c7370e7..46fa8d9dc66c 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 @@ -465,9 +465,7 @@ protected List balanceTable(TableName tableName, // Allow turning this feature off if the locality cost is not going to // be used in any computations. RegionHDFSBlockLocationFinder finder = null; - if ( - (this.localityCost != null) || (this.rackLocalityCost != null) - ) { + if ((this.localityCost != null) || (this.rackLocalityCost != null)) { finder = this.regionFinder; } From 8645e930f2e34c4a017c766b01ff5155eed29048 Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Fri, 3 Mar 2023 18:08:23 +0530 Subject: [PATCH 08/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions Updated javadoc comments. --- .../master/balancer/PrefetchCacheCostFunction.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java index 225b97b30e1a..3c897f7aea5c 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -36,12 +36,12 @@ public class PrefetchCacheCostFunction extends CostFunction { private float bestPrefetchRatio; /** - * Enables or disables the prefetch cache cost function depending on the parameter - * PREFETCH_PERSISTENCE_PATH_KEY. If set, this parameter enables the prefetched file list - * persistence. If this parameter is not set this means that the cache persistence is disabled - * which means that the prefetch ratios of regions on region servers cannot be calculated and - * hence the regions should be moved based on how much they have been prefetched on a region - * server. The prefetch cache cost function is disabled if the multiplier is set to 0. + * The prefetch cache cost function is enabled only when the prefetch file list persistence is + * enabled by setting the parameter PREFETCH_PERSISTENCE_PATH_KEY. The prefetch file list + * persistence is disabled by default. The prefetch cache cost function is also disabled if the + * multiplier is set to 0. The prefetch cache ratio function would be most relevant for non-hdfs + * deployments, which then makes locality irrelevant. In those cases, prefetch and region skewness + * would be competing to prevail over the final balancer decision. * @param conf Cluster configuration */ PrefetchCacheCostFunction(Configuration conf) { From 3fcaa560df62e20ec97b266eb6cd7274c43d5df7 Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Fri, 24 Mar 2023 18:03:28 +0530 Subject: [PATCH 09/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions Fixed following issues found during testing 1. The historical prefetch ratio was maintained in a map using the server name in the format . The startcode changes with every server restart which means that the server name will be different after each server restart. This can cause issue while calculating the historical prefetch ratio as the balancer may not be able to find the historical prefetch because of the server name mismatch. 2. Fixed an issue in the region server which may lead to incorrect prefetch ratio calculation if the HFile in a region is a link 3. Added a system test to test the changes --- .../master/balancer/BalancerClusterState.java | 14 +- .../master/balancer/BalancerRegionLoad.java | 5 +- .../balancer/PrefetchCacheCostFunction.java | 27 +- .../balancer/StochasticLoadBalancer.java | 13 +- ...PrefetchCacheCostLoadBalancerFunction.java | 18 +- .../apache/hadoop/hbase/RegionMetrics.java | 2 +- .../hadoop/hbase/RegionMetricsBuilder.java | 14 +- .../main/protobuf/server/ClusterStatus.proto | 2 +- .../hbase/regionserver/HRegionServer.java | 15 +- .../master/TestRegionsRecoveryChore.java | 2 +- .../TestPrefetchCacheCostBalancer.java | 321 ++++++++++++++++++ 11 files changed, 387 insertions(+), 46 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.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 ee852a86fe51..03237c30a255 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 @@ -121,7 +121,7 @@ class BalancerClusterState { // Maps region -> serverIndex with best prefect ratio private int[] regionServerIndexWithBestPrefetchRatio; // Historical region server prefetch ratio - Map> historicalRegionServerPrefetchRatio; + Map> historicalRegionServerPrefetchRatio; static class DefaultRackManager extends RackManager { @Override @@ -138,7 +138,7 @@ public String getRack(ServerName server) { BalancerClusterState(Map> clusterState, Map> loads, RegionHDFSBlockLocationFinder regionFinder, - RackManager rackManager, Map> oldRegionServerPrefetchRatio) { + RackManager rackManager, Map> oldRegionServerPrefetchRatio) { this(null, clusterState, loads, regionFinder, rackManager, oldRegionServerPrefetchRatio); } @@ -146,7 +146,7 @@ public String getRack(ServerName server) { BalancerClusterState(Collection unassignedRegions, Map> clusterState, Map> loads, RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager, - Map> oldRegionServerPrefetchRatio) { + Map> oldRegionServerPrefetchRatio) { if (unassignedRegions == null) { unassignedRegions = Collections.emptyList(); } @@ -611,15 +611,15 @@ protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) // Seartch 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 regionNameAsString = regions[region].getRegionNameAsString(); - String serverNameAsString = servers[regionServerIndex].getServerName(); + Address serverAddress = servers[regionServerIndex].getAddress(); if ( historicalRegionServerPrefetchRatio != null && historicalRegionServerPrefetchRatio.containsKey(regionNameAsString) ) { - Map serverPrefetchRatio = + Map serverPrefetchRatio = historicalRegionServerPrefetchRatio.get(regionNameAsString); - if (serverPrefetchRatio.containsKey(serverNameAsString)) { - prefetchRatio = serverPrefetchRatio.get(serverNameAsString); + if (serverPrefetchRatio.containsKey(serverAddress)) { + prefetchRatio = serverPrefetchRatio.get(serverAddress); // The old prefetch cache ratio has been accounted for and hence, clear up this information historicalRegionServerPrefetchRatio.remove(regionNameAsString, serverPrefetchRatio); 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 c925d37d3a19..4728854d323e 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.balancer; import org.apache.hadoop.hbase.RegionMetrics; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Size; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; @@ -35,7 +36,7 @@ class BalancerRegionLoad { private final int memStoreSizeMB; private final int storefileSizeMB; private final float prefetchCacheRatio; - private final String serverName; + private final ServerName serverName; BalancerRegionLoad(RegionMetrics regionMetrics) { readRequestsCount = regionMetrics.getReadRequestCount(); @@ -71,7 +72,7 @@ public float getPrefetchCacheRatio() { return prefetchCacheRatio; } - public String getServerName() { + public ServerName getServerName() { return serverName; } } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java index 3c897f7aea5c..349208dfa23c 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -20,6 +20,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Compute the cost of a potential cluster configuration based on the number of HFile's already @@ -27,13 +29,14 @@ */ @InterfaceAudience.Private public class PrefetchCacheCostFunction extends CostFunction { + private static final Logger LOG = LoggerFactory.getLogger(PrefetchCacheCostFunction.class); private static final String PREFETCH_CACHE_COST_KEY = "hbase.master.balancer.stochastic.prefetchCacheCost"; private static final float DEFAULT_PREFETCH_COST = 500; private String prefetchedFileListPath; private double prefetchRatio; - private float bestPrefetchRatio; + private double bestPrefetchRatio; /** * The prefetch cache cost function is enabled only when the prefetch file list persistence is @@ -49,15 +52,15 @@ public class PrefetchCacheCostFunction extends CostFunction { this.setMultiplier(prefetchedFileListPath == null ? 0.0f : conf.getFloat(PREFETCH_CACHE_COST_KEY, DEFAULT_PREFETCH_COST)); - prefetchRatio = 0.0f; - bestPrefetchRatio = 0.0f; + prefetchRatio = 0.0; + bestPrefetchRatio = 0.0; } @Override void prepare(BalancerClusterState cluster) { super.prepare(cluster); - prefetchRatio = 0.0f; - bestPrefetchRatio = 0.0f; + prefetchRatio = 0.0; + bestPrefetchRatio = 0.0; for (int region = 0; region < cluster.numRegions; region++) { prefetchRatio += @@ -65,7 +68,7 @@ void prepare(BalancerClusterState cluster) { bestPrefetchRatio += cluster.getOrComputeWeightedPrefetchRatio(region, cluster.getOrComputeServerWithBestPrefetchRatio()[region]); } - prefetchRatio = bestPrefetchRatio == 0.0f ? 1.0f : prefetchRatio / bestPrefetchRatio; + prefetchRatio = bestPrefetchRatio == 0.0 ? 1.0 : prefetchRatio / bestPrefetchRatio; } @Override @@ -75,11 +78,15 @@ protected double cost() { @Override protected void regionMoved(int region, int oldServer, int newServer) { - float oldServerPrefetch = cluster.getOrComputeWeightedPrefetchRatio(region, oldServer); - float newServerPrefetch = cluster.getOrComputeWeightedPrefetchRatio(region, newServer); - float prefetchDelta = newServerPrefetch - oldServerPrefetch; - float normalizeDelta = bestPrefetchRatio == 0.0f ? 0.0f : prefetchDelta / bestPrefetchRatio; + double oldServerPrefetch = cluster.getOrComputeWeightedPrefetchRatio(region, oldServer); + double newServerPrefetch = cluster.getOrComputeWeightedPrefetchRatio(region, newServer); + double prefetchDelta = newServerPrefetch - oldServerPrefetch; + double normalizeDelta = bestPrefetchRatio == 0.0 ? 0.0 : prefetchDelta / bestPrefetchRatio; prefetchRatio += normalizeDelta; + LOG.debug( + "Region {} moved from {} to {} with oldServerPrefetch {}," + + " newServerPrefetch {} and bestPrefetchRatio {}", + region, oldServer, newServer, oldServerPrefetch, newServerPrefetch, bestPrefetchRatio); } @Override 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 46fa8d9dc66c..ceee528dd308 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 @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.master.RackManager; import org.apache.hadoop.hbase.master.RegionPlan; +import org.apache.hadoop.hbase.net.Address; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; @@ -132,8 +133,8 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { Map> loads = new HashMap<>(); // Map of old prefetch ratio (region name ---> old server name ---> old prefetch ratio) - Map> historicRegionServerPrefetchRatio = - new HashMap>(); + Map> historicRegionServerPrefetchRatio = + new HashMap>(); // values are defaults private int maxSteps = DEFAULT_MAX_STEPS; @@ -734,7 +735,7 @@ private void updateRegionLoad() { sm.getRegionMetrics().forEach((byte[] regionName, RegionMetrics rm) -> { String regionNameAsString = RegionInfo.getRegionNameAsString(regionName); Deque rLoads = oldLoads.get(regionNameAsString); - String oldServerName = null; + ServerName oldServerName = null; float oldPrefetchRatio = 0.0f; if (rLoads == null) { rLoads = new ArrayDeque<>(numRegionLoadsToRemember + 1); @@ -746,10 +747,10 @@ private void updateRegionLoad() { rLoads.remove(); } } - if (oldServerName != null && !oldServerName.equals(rm.getServerName())) { + if (oldServerName != null && !ServerName.isSameAddress(oldServerName, rm.getServerName())) { // Record the old region server prefetch ratio - Map serverPrefetchRatio = new HashMap<>(); - serverPrefetchRatio.put(oldServerName, oldPrefetchRatio); + Map serverPrefetchRatio = new HashMap<>(); + serverPrefetchRatio.put(oldServerName.getAddress(), oldPrefetchRatio); historicRegionServerPrefetchRatio.put(regionNameAsString, serverPrefetchRatio); } rLoads.add(new BalancerRegionLoad(rm)); diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java index ceb5d5c769a1..e1e5272dd4f1 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java @@ -27,6 +27,7 @@ import java.util.Map; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.net.Address; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.ClassRule; @@ -219,8 +220,8 @@ public MockClusterForPrefetch(int[][] regionsArray) { // regions[0] is an array where index = serverIndex and value = number of regions super(mockClusterServers(regionsArray[0], 1), null, null, null, null); regionServerPrefetch = new int[regionsArray.length - 1][]; - Map> historicalPrefetchRatio = - new HashMap>(); + Map> historicalPrefetchRatio = + new HashMap>(); for (int i = 1; i < regionsArray.length; i++) { int regionIndex = i - 1; regionServerPrefetch[regionIndex] = new int[regionsArray[i].length - 1]; @@ -230,9 +231,8 @@ public MockClusterForPrefetch(int[][] regionsArray) { regionServerPrefetch[regionIndex][serverIndex] = regionsArray[i][j]; if (regionsArray[i][j] > 0 && serverIndex != regionsArray[i][0]) { // This is the historical prefetch value - Map historicalPrefetch = new HashMap<>(); - historicalPrefetch.put(servers[serverIndex].getServerName(), - (float) regionsArray[i][j]); + Map historicalPrefetch = new HashMap<>(); + historicalPrefetch.put(servers[serverIndex].getAddress(), (float) regionsArray[i][j]); historicalPrefetchRatio.put(regions[regionIndex].getRegionNameAsString(), historicalPrefetch); } @@ -260,15 +260,15 @@ protected float getRegionServerPrefetchRatio(int region, int regionServerIndex) // 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 regionNameAsString = regions[region].getRegionNameAsString(); - String serverNameAsString = servers[regionServerIndex].getServerName(); + Address serverAddress = servers[regionServerIndex].getAddress(); if ( historicalRegionServerPrefetchRatio != null && historicalRegionServerPrefetchRatio.containsKey(regionNameAsString) ) { - Map serverPrefetchRatio = + Map serverPrefetchRatio = historicalRegionServerPrefetchRatio.get(regionNameAsString); - if (serverPrefetchRatio.containsKey(serverNameAsString)) { - prefetchRatio = serverPrefetchRatio.get(serverNameAsString); + if (serverPrefetchRatio.containsKey(serverAddress)) { + prefetchRatio = serverPrefetchRatio.get(serverAddress); // The old prefetch cache ratio has been accounted for and hence, clear up this // information // as it is not needed anymore diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java index 0ed25f123005..285aa50b1143 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java @@ -143,5 +143,5 @@ default String getNameAsString() { float getPrefetchCacheRatio(); /** Returns the name of the server hosting this region */ - String getServerName(); + ServerName getServerName(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java index 102cc1d56d7a..2a34ba227c06 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java @@ -81,7 +81,7 @@ public static RegionMetrics toRegionMetrics(ClusterStatusProtos.RegionLoad regio .setUncompressedStoreFileSize( new Size(regionLoadPB.getStoreUncompressedSizeMB(), Size.Unit.MEGABYTE)) .setPrefetchCacheRatio(regionLoadPB.getPrefetchCacheRatio()) - .setServerName(regionLoadPB.getServerName()).build(); + .setServerName(ProtobufUtil.toServerName(regionLoadPB.getServerName())).build(); } private static List @@ -122,7 +122,7 @@ public static ClusterStatusProtos.RegionLoad toRegionLoad(RegionMetrics regionMe .setStoreUncompressedSizeMB( (int) regionMetrics.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE)) .setPrefetchCacheRatio(regionMetrics.getPrefetchCacheRatio()) - .setServerName(regionMetrics.getServerName()).build(); + .setServerName(ProtobufUtil.toServerName(regionMetrics.getServerName())).build(); } public static RegionMetricsBuilder newBuilder(byte[] name) { @@ -157,7 +157,7 @@ public static RegionMetricsBuilder newBuilder(byte[] name) { private long blocksTotalWeight; private CompactionState compactionState; private float prefetchCacheRatio; - private String serverName; + private ServerName serverName; private RegionMetricsBuilder(byte[] name) { this.name = name; @@ -298,7 +298,7 @@ public RegionMetricsBuilder setPrefetchCacheRatio(float prefetchCacheRatio) { return this; } - public RegionMetricsBuilder setServerName(String serverName) { + public RegionMetricsBuilder setServerName(ServerName serverName) { this.serverName = serverName; return this; } @@ -342,7 +342,7 @@ private static class RegionMetricsImpl implements RegionMetrics { private final long blocksTotalWeight; private final CompactionState compactionState; private final float prefetchCacheRatio; - private final String serverName; + private final ServerName serverName; RegionMetricsImpl(byte[] name, int storeCount, int storeFileCount, int storeRefCount, int maxCompactedStoreFileRefCount, final long compactingCellCount, long compactedCellCount, @@ -352,7 +352,7 @@ private static class RegionMetricsImpl implements RegionMetrics { long filteredReadRequestCount, long completedSequenceId, Map storeSequenceIds, float dataLocality, long lastMajorCompactionTimestamp, float dataLocalityForSsd, long blocksLocalWeight, long blocksLocalWithSsdWeight, long blocksTotalWeight, - CompactionState compactionState, float prefetchCacheRatio, String serverName) { + CompactionState compactionState, float prefetchCacheRatio, ServerName serverName) { this.name = Preconditions.checkNotNull(name); this.storeCount = storeCount; this.storeFileCount = storeFileCount; @@ -525,7 +525,7 @@ public float getPrefetchCacheRatio() { } @Override - public String getServerName() { + public ServerName getServerName() { return serverName; } diff --git a/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto b/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto index b994308e966e..e86f463c7fc6 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/ClusterStatus.proto @@ -182,7 +182,7 @@ message RegionLoad { optional float prefetch_cache_ratio = 28; /** The server hosting this region */ - optional string server_name = 29; + optional ServerName server_name = 29; } message UserLoad { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 807080b4cfdd..124ab3031f77 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -106,6 +106,7 @@ import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; import org.apache.hadoop.hbase.executor.ExecutorType; import org.apache.hadoop.hbase.http.InfoServer; +import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory; import org.apache.hadoop.hbase.io.hfile.HFile; @@ -1556,7 +1557,16 @@ RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, Collection filesInStore = store.getStorefiles(); if (!filesInStore.isEmpty()) { for (HStoreFile hStoreFile : filesInStore) { - if (PrefetchExecutor.isFilePrefetched(hStoreFile.getPath().getName())) { + String tempFileName = hStoreFile.getPath().getName(); + // PrefetchExecutor adds the encoded HFile name if the file has been prefetched. Every + // store file in this region is then checked if it has been prefetched. If the storefile + // is a link, then it's format is . The comparison here would fail if + // this format is used. In order for this comparison to success in such case, the HFile + // name needs to be extracted from the HFileLink + if (HFileLink.isHFileLink(tempFileName)) { + tempFileName = HFileLink.getReferencedHFileName(tempFileName); + } + if (PrefetchExecutor.isFilePrefetched(tempFileName)) { filesAlreadyPrefetched++; } } @@ -1608,7 +1618,8 @@ RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, .setBlocksLocalWithSsdWeight(blocksLocalWithSsdWeight).setBlocksTotalWeight(blocksTotalWeight) .setCompactionState(ProtobufUtil.createCompactionStateForRegionLoad(r.getCompactionState())) .setLastMajorCompactionTs(r.getOldestHfileTs(true)) - .setPrefetchCacheRatio(ratioOfFilesAlreadyCached).setServerName(serverName.getServerName()); + .setPrefetchCacheRatio(ratioOfFilesAlreadyCached) + .setServerName(ProtobufUtil.toServerName(serverName)); r.setCompleteSequenceId(regionLoadBldr); return regionLoadBldr.build(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java index 9f8b86c6c0e8..ea4abc0c422b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java @@ -548,7 +548,7 @@ public float getPrefetchCacheRatio() { } @Override - public String getServerName() { + public ServerName getServerName() { return null; } }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java new file mode 100644 index 000000000000..5da027440846 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java @@ -0,0 +1,321 @@ +/* + * 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.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerMetrics; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.SingleProcessHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.io.ByteBuffAllocator; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Test prefetchCacheCostFunction balancer function + */ +@Category({ MasterTests.class, MediumTests.class }) +public class TestPrefetchCacheCostBalancer extends StochasticBalancerTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestPrefetchCacheCostBalancer.class); + + private static HBaseTestingUtil TEST_UTIL; + + private static final int REGION_SERVERS = 3; + + private static final int REGION_NUM = REGION_SERVERS * 3; + + private Admin admin; + + private SingleProcessHBaseCluster cluster; + + private Configuration conf; + + private enum FunctionCostKeys { + REGIONCOUNTCOST("hbase.master.balancer.stochastic.regionCountCost"), + PRIMARYREGIONCOUNTCOST("hbase.master.balancer.stochastic.primaryRegionCountCost"), + MOVECOST("hbase.master.balancer.stochastic.moveCost"), + LOCALITYCOST("hbase.master.balancer.stochastic.localityCost"), + RACKLOCALITYCOST("hbase.master.balancer.stochastic.rackLocalityCost"), + TABLESKEWCOST("hbase.master.balancer.stochastic.tableSkewCost"), + REGIONREPLICAHOSTCOSTKEY("hbase.master.balancer.stochastic.regionReplicaHostCostKey"), + REGIONREPLICARACKCOSTKEY("hbase.master.balancer.stochastic.regionReplicaRackCostKey"), + READREQUESTCOST("hbase.master.balancer.stochastic.readRequestCost"), + CPREQUESTCOST("hbase.master.balancer.stochastic.cpRequestCost"), + WRITEREQUESTCOST("hbase.master.balancer.stochastic.writeRequestCost"), + MEMSTORESIZECOST("hbase.master.balancer.stochastic.memstoreSizeCost"), + STOREFILESIZECOST("hbase.master.balancer.stochastic.storefileSizeCost"); + + private final String costKey; + + FunctionCostKeys(String key) { + this.costKey = key; + } + + public String getValue() { + return costKey; + } + } + + @Before + public void setup() throws Exception { + TEST_UTIL = new HBaseTestingUtil(); + conf = TEST_UTIL.getConfiguration(); + Path testDir = TEST_UTIL.getDataTestDir(); + + // Enable prefetch persistence which will enable prefetch cache cost function + Path p = new Path(testDir, "bc.txt"); + FileSystem fs = FileSystem.get(this.conf); + fs.create(p).close(); + // Must use file based bucket cache here. + conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "file:" + p); + String prefetchPersistencePath = testDir + "/prefetch.persistence"; + conf.setBoolean(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, true); + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, prefetchPersistencePath); + // Must use the ByteBuffAllocator here + conf.setBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, true); + conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.1f); + conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap"); + // 32MB for BucketCache. + conf.setFloat(HConstants.BUCKET_CACHE_SIZE_KEY, 32); + } + + @After + public void cleanup() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + TEST_UTIL.cleanupTestDir(); + } + + // Verify whether the given cost function is enabled/disabled. A function is enabled if the + // cost key is set to a positive number and disabled if it is set to 0 + private void verifyCostFunctionState(Configuration myConf, String costFunctionName, + boolean isEnabled) { + float costKeyValue = myConf.getFloat(costFunctionName, Float.MAX_VALUE); + assertEquals(isEnabled, costKeyValue > 0.0f); + } + + @Test + public void testOnlyPrefetchCacheCostFunctionDisabled() throws Exception { + // Test strategy + // 1. Turn off the prefetch cache cost function by setting the parameter + // hbase.master.balancer.stochastic.prefetchCacheCost to 0 + // 2. Create the cluster + // 3. Find a region server to shutdown and restart + // 4. Assert that the region server identified in 3. has more than 1 regions assigned + // 5. Shutdown the region server + // 6. Get the number of regions assigned to the other region server and assert that it matched + // the total number of regions in the cluster + // 7. Start the region server identified in 3. + // 8. Trigger the balancer + // 9. Assert that some regions are assigned to the region server identified in 3. + + // Disable the prefetch cache cost function + conf.setFloat("hbase.master.balancer.stochastic.prefetchCacheCost", 0.0f); + + TEST_UTIL.startMiniCluster(REGION_SERVERS); + TEST_UTIL.getDFSCluster().waitClusterUp(); + + cluster = TEST_UTIL.getHBaseCluster(); + admin = TEST_UTIL.getAdmin(); + admin.balancerSwitch(false, true); + TableName tableName = TableName.valueOf("testTablePrefetchCacheCostFunctionDisabled"); + TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build(); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TEST_UTIL.waitTableAvailable(tableName); + TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); + admin.flush(tableName); + compactTable(tableName); + + // Validate that all the other cost functions are enabled + Arrays.stream(FunctionCostKeys.values()) + .forEach(functionCostKey -> verifyCostFunctionState(admin.getConfiguration(), + functionCostKey.getValue(), true)); + + // Validate that the prefetch cache cost function is disabled + verifyCostFunctionState(admin.getConfiguration(), + "hbase.master.balancer.stochastic.prefetchCacheCost", false); + + // Run the balancer and wait until all the region movement is finished + admin.balancerSwitch(true, true); + assertTrue("Balancer did not run", admin.balance()); + admin.balance(); + TEST_UTIL.waitUntilNoRegionsInTransition(120000); + + Map ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + assertEquals(REGION_SERVERS, ssmap.size()); + + // Get the name of the region server to shutdown and restart + ServerName serverName = cluster.getRegionServer(REGION_SERVERS - 1).getServerName(); + ServerMetrics sm = ssmap.get(serverName); + // Verify that some regions are assigned to this region server + assertTrue(0.0f != sm.getRegionMetrics().size()); + + // Shutdown the region server and wait for the regions hosted by this server to be reassigned + // to other region servers + cluster.stopRegionServer(serverName); + cluster.waitForRegionServerToStop(serverName, 1000); + // Compact the table so that all the regions are reassigned to the running region servers + compactTable(tableName); + TEST_UTIL.waitUntilNoRegionsInTransition(12000); + + ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + assertEquals(REGION_SERVERS - 1, ssmap.size()); + sm = ssmap.get(serverName); + // Validate that no server metrics is found for the non-active server + assertNull(sm); + + // Restart the region server and run balancer and validate that some regions are reassigned to + // this region server + cluster.startRegionServer(serverName.getHostname(), serverName.getPort()); + // Get the name of the region server + cluster.waitForRegionServerToStart(serverName.getHostname(), serverName.getPort(), 1000); + admin.balance(); + TEST_UTIL.waitUntilNoRegionsInTransition(12000); + ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + assertEquals(REGION_SERVERS, ssmap.size()); + + serverName = cluster.getRegionServer(REGION_SERVERS - 1).getServerName(); + sm = ssmap.get(serverName); + + assertNotNull(sm); + assertTrue(sm.getRegionMetrics().size() > 0); + } + + @Test + public void testOnlyPrefetchCacheCostFunctionEnabled() throws Exception { + // Test strategy + // 1. Turn off all other cost functions. NOTE: Please add to the list of cost functions that + // need + // to be turned off when a new function is added + // 2. Create a cluster only with prefetchCacheCostFunction enabled + // 3. Find a regionserver to shutdown and restart + // 4. Assert that the region server identified in 3. has more than 1 regions assigned + // 5. Shutdown the region server identified in 3. + // 6. Get the number of regions assigned to the other region servers and assert that it matches + // the total number of regions in the cluster + // 7. Start the region server identified in 3. + // 8. Trigger the balancer + // 9. Assert that no regions are assigned to the region server identified in 3. + + Arrays.stream(FunctionCostKeys.values()) + .forEach(functionCostKey -> conf.setFloat(functionCostKey.getValue(), 0.0f)); + + TEST_UTIL.startMiniCluster(REGION_SERVERS); + TEST_UTIL.getDFSCluster().waitClusterUp(); + cluster = TEST_UTIL.getHBaseCluster(); + admin = TEST_UTIL.getAdmin(); + admin.balancerSwitch(false, true); + TableName tableName = TableName.valueOf("testTableOnlyPrefetchCacheCostFunctionEnabled"); + TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build(); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TEST_UTIL.waitTableAvailable(tableName); + TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); + admin.flush(tableName); + compactTable(tableName); + + // Validate that all the other cost functions are disabled + Arrays.stream(FunctionCostKeys.values()) + .forEach(functionCostKey -> verifyCostFunctionState(admin.getConfiguration(), + functionCostKey.getValue(), false)); + + verifyCostFunctionState(admin.getConfiguration(), + "hbase.master.balancer.stochastic.prefetchCacheCost", true); + + // Run balancer and wait until all the region have been moved + // started. + admin.balancerSwitch(true, true); + assertTrue("Balancer did not run", admin.balance()); + admin.balance(); + TEST_UTIL.waitUntilNoRegionsInTransition(120000); + + Map ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + + assertEquals(REGION_SERVERS, ssmap.size()); + + // Shutdown the last server. This is because the server id for an inactive server is reassigned + // to the next running server. As soon as this server is restarted, it is assigned the next + // available + // server id. In our case, we want to track the same server and hence, it's safe to restart the + // last server in the list + ServerName serverName = cluster.getRegionServer(REGION_SERVERS - 1).getServerName(); + ServerMetrics sm = ssmap.get(serverName); + assertTrue(0 != sm.getRegionMetrics().size()); + + cluster.stopRegionServer(serverName); + cluster.waitForRegionServerToStop(serverName, 1000); + compactTable(tableName); + TEST_UTIL.waitUntilNoRegionsInTransition(12000); + ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + assertEquals(REGION_SERVERS - 1, ssmap.size()); + sm = ssmap.get(serverName); + assertNull(sm); + + // Restart the region server + cluster.startRegionServer(serverName.getHostname(), serverName.getPort()); + cluster.waitForRegionServerToStart(serverName.getHostname(), serverName.getPort(), 1000); + admin.balance(); + TEST_UTIL.waitUntilNoRegionsInTransition(120000); + ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + assertEquals(REGION_SERVERS, ssmap.size()); + + serverName = cluster.getRegionServer(REGION_SERVERS - 1).getServerName(); + sm = ssmap.get(serverName); + + assertNotNull(sm); + assertEquals(0, sm.getRegionMetrics().size()); + } + + private void compactTable(TableName tableName) throws IOException { + for (JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) { + for (HRegion region : t.getRegionServer().getRegions(tableName)) { + region.compact(true); + region.flush(true); + } + } + } +} From af0181bf6a20b6edd572d63f5a2a567a80088baf Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Fri, 24 Mar 2023 19:34:40 +0530 Subject: [PATCH 10/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions Fixed build failures --- .../master/balancer/PrefetchBasedCandidateGenerator.java | 2 -- .../TestPrefetchCacheCostLoadBalancerFunction.java | 7 ++----- .../hbase/io/hfile/bucket/TestBucketCachePersister.java | 3 ++- 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java index 3244973fbcad..b6e2b51ab42c 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java @@ -24,8 +24,6 @@ @InterfaceAudience.Private class PrefetchBasedCandidateGenerator extends CandidateGenerator { - private static final Logger LOG = LoggerFactory.getLogger(PrefetchBasedCandidateGenerator.class); - private static float PREFETCH_RATIO_DIFF_FACTOR = 1.25f; @Override diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java index e1e5272dd4f1..9f81169e64e0 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.master.balancer; -import static org.apache.hadoop.hbase.master.balancer.StochasticBalancerTestBase.loadBalancer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -41,17 +40,15 @@ public class TestPrefetchCacheCostLoadBalancerFunction extends StochasticBalance public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestPrefetchCacheCostLoadBalancerFunction.class); - private static final String REGION_KEY = "testRegion"; - // Mapping of prefetch test -> expected prefetch - private float[] expectedPrefetch = { 0.0f, 0.0f, 0.5f, 1.0f, 0.0f, 0.572f, 0.0f, 0.075f }; + 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 int[][][] clusterRegionPrefetchMocks = new int[][][] { + 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 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCachePersister.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCachePersister.java index 9e90d0e229c6..5b8d305e6087 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCachePersister.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCachePersister.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; @@ -84,7 +85,7 @@ public Configuration setupBucketCacheConfig(long bucketCachePersistInterval) thr } public BucketCache setupBucketCache(Configuration conf) throws IOException { - conf.set(CacheConfig.PREFETCH_PERSISTENCE_PATH_KEY, (testDir + "/prefetch.persistence")); + conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, (testDir + "/prefetch.persistence")); BucketCache bucketCache = new BucketCache("file:" + testDir + "/bucket.cache", capacitySize, constructedBlockSize, constructedBlockSizes, writeThreads, writerQLen, testDir + "/bucket.persistence", 60 * 1000, conf); From cf7b3d2d536beec2a4b7dae9a85873c84118df96 Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Sat, 25 Mar 2023 00:17:06 +0530 Subject: [PATCH 11/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions --- .../PrefetchBasedCandidateGenerator.java | 2 -- .../balancer/PrefetchCacheCostFunction.java | 5 ----- .../apache/hadoop/hbase/TestServerMetrics.java | 17 +++++++++++------ 3 files changed, 11 insertions(+), 13 deletions(-) diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java index b6e2b51ab42c..6d7c7ed5b88c 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchBasedCandidateGenerator.java @@ -19,8 +19,6 @@ import java.util.concurrent.ThreadLocalRandom; import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; @InterfaceAudience.Private class PrefetchBasedCandidateGenerator extends CandidateGenerator { diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java index 349208dfa23c..91d5d58868e9 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -29,7 +29,6 @@ */ @InterfaceAudience.Private public class PrefetchCacheCostFunction extends CostFunction { - private static final Logger LOG = LoggerFactory.getLogger(PrefetchCacheCostFunction.class); private static final String PREFETCH_CACHE_COST_KEY = "hbase.master.balancer.stochastic.prefetchCacheCost"; private static final float DEFAULT_PREFETCH_COST = 500; @@ -83,10 +82,6 @@ protected void regionMoved(int region, int oldServer, int newServer) { double prefetchDelta = newServerPrefetch - oldServerPrefetch; double normalizeDelta = bestPrefetchRatio == 0.0 ? 0.0 : prefetchDelta / bestPrefetchRatio; prefetchRatio += normalizeDelta; - LOG.debug( - "Region {} moved from {} to {} with oldServerPrefetch {}," - + " newServerPrefetch {} and bestPrefetchRatio {}", - region, oldServer, newServer, oldServerPrefetch, newServerPrefetch, bestPrefetchRatio); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java index 8bcf3e600f88..f0c7cb72f863 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java @@ -28,6 +28,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; @@ -58,6 +59,8 @@ public void testRegionLoadAggregation() { metrics.getRegionMetrics().values().stream().mapToLong(v -> v.getReadRequestCount()).sum()); assertEquals(100, metrics.getRegionMetrics().values().stream().mapToLong(v -> v.getCpRequestCount()).sum()); + assertEquals(1.0, metrics.getRegionMetrics().values().stream() + .mapToDouble(v -> v.getPrefetchCacheRatio()).sum(), 0); assertEquals(300, metrics.getRegionMetrics().values().stream() .mapToLong(v -> v.getFilteredReadRequestCount()).sum()); } @@ -95,16 +98,18 @@ private ClusterStatusProtos.ServerLoad createServerLoadProto() { .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME) .setValue(ByteString.copyFromUtf8("QWERTYUIOP")).build(); - ClusterStatusProtos.RegionLoad rlOne = - ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecOne).setStores(10) - .setStorefiles(101).setStoreUncompressedSizeMB(106).setStorefileSizeMB(520) - .setFilteredReadRequestsCount(100).setStorefileIndexSizeKB(42).setRootIndexSizeKB(201) - .setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build(); + ClusterStatusProtos.RegionLoad rlOne = ClusterStatusProtos.RegionLoad.newBuilder() + .setRegionSpecifier(rSpecOne).setStores(10).setStorefiles(101).setStoreUncompressedSizeMB(106) + .setStorefileSizeMB(520).setFilteredReadRequestsCount(100).setStorefileIndexSizeKB(42) + .setRootIndexSizeKB(201).setReadRequestsCount(Integer.MAX_VALUE) + .setWriteRequestsCount(Integer.MAX_VALUE).setPrefetchCacheRatio(0.0f) + .setServerName(ProtobufUtil.toServerName(new ServerName("localhost1", 1, 1))).build(); ClusterStatusProtos.RegionLoad rlTwo = ClusterStatusProtos.RegionLoad.newBuilder() .setRegionSpecifier(rSpecTwo).setStores(3).setStorefiles(13).setStoreUncompressedSizeMB(23) .setStorefileSizeMB(300).setFilteredReadRequestsCount(200).setStorefileIndexSizeKB(40) .setRootIndexSizeKB(303).setReadRequestsCount(Integer.MAX_VALUE) - .setWriteRequestsCount(Integer.MAX_VALUE).setCpRequestsCount(100).build(); + .setWriteRequestsCount(Integer.MAX_VALUE).setCpRequestsCount(100).setPrefetchCacheRatio(1.0f) + .setServerName(ProtobufUtil.toServerName(new ServerName("localhost2", 1, 1))).build(); ClusterStatusProtos.ServerLoad sl = ClusterStatusProtos.ServerLoad.newBuilder() .addRegionLoads(rlOne).addRegionLoads(rlTwo).build(); From 1a7d728b3cae60df967dd4fef899a8db3d662ef9 Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Mon, 27 Mar 2023 18:49:26 +0530 Subject: [PATCH 12/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions --- .../hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java index 91d5d58868e9..f19fc88618e7 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -20,8 +20,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Compute the cost of a potential cluster configuration based on the number of HFile's already From 160f48435281c5479c4ea5ea8cbabc6815846607 Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Fri, 31 Mar 2023 11:05:04 +0530 Subject: [PATCH 13/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions --- .../hbase/master/balancer/PrefetchCacheCostFunction.java | 2 +- .../hbase/master/balancer/TestPrefetchCacheCostBalancer.java | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java index f19fc88618e7..7e7778597842 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/balancer/PrefetchCacheCostFunction.java @@ -70,7 +70,7 @@ void prepare(BalancerClusterState cluster) { @Override protected double cost() { - return 1 - prefetchRatio; + return scale(0, 1, (1 - prefetchRatio)); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java index 5da027440846..dafb5b965827 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java @@ -185,7 +185,7 @@ public void testOnlyPrefetchCacheCostFunctionDisabled() throws Exception { admin.balance(); TEST_UTIL.waitUntilNoRegionsInTransition(120000); - Map ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + Map ssmap = cluster.getClusterMetrics().getLiveServerMetrics(); assertEquals(REGION_SERVERS, ssmap.size()); // Get the name of the region server to shutdown and restart @@ -273,8 +273,7 @@ public void testOnlyPrefetchCacheCostFunctionEnabled() throws Exception { admin.balance(); TEST_UTIL.waitUntilNoRegionsInTransition(120000); - Map ssmap = admin.getClusterMetrics().getLiveServerMetrics(); - + Map ssmap = cluster.getClusterMetrics().getLiveServerMetrics(); assertEquals(REGION_SERVERS, ssmap.size()); // Shutdown the last server. This is because the server id for an inactive server is reassigned From 55e55ea3c03a363732779118f3c5a229b819438c Mon Sep 17 00:00:00 2001 From: Rahul Agarkar Date: Fri, 31 Mar 2023 19:06:07 +0530 Subject: [PATCH 14/14] HBASE-27389 Add cost function in balancer to consider the cost of building bucket cache before moving regions --- .../balancer/StochasticLoadBalancer.java | 6 + ...PrefetchCacheCostLoadBalancerFunction.java | 34 ++++- .../TestPrefetchCacheCostBalancer.java | 120 ++++++++++++++---- 3 files changed, 132 insertions(+), 28 deletions(-) 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 ceee528dd308..bbe85c6af0bc 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 @@ -770,6 +770,12 @@ void initCosts(BalancerClusterState cluster) { } } + @RestrictedApi(explanation = "Should only be called in tests", link = "", + allowedOnPath = ".*(/src/test/.*|StochasticLoadBalancer).java") + List getCostFunctions() { + return costFunctions; + } + /** * Update both the costs of costfunctions and the weights of candidate generators */ diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java index 9f81169e64e0..575def6f6b9d 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostLoadBalancerFunction.java @@ -24,11 +24,14 @@ 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.net.Address; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.Before; +import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -177,6 +180,19 @@ public class TestPrefetchCacheCostLoadBalancerFunction extends StochasticBalance // No historical prefetch }, }; + private static Configuration storedConfiguration; + + @BeforeClass + public static void saveInitialConfiguration() { + storedConfiguration = new Configuration(conf); + } + + @Before + public void beforeEachTest() { + conf = new Configuration(storedConfiguration); + loadBalancer.onConfigurationChange(conf); + } + @Test public void testVerifyPrefetchCostFunctionEnabled() { conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch.persistence"); @@ -189,13 +205,22 @@ public void testVerifyPrefetchCostFunctionEnabled() { } @Test - public void testVerifyPrefetchCostFunctionDisabled() { + public void testVerifyPrefetchCostFunctionDisabledByNoPersistencePathKey() { + assertFalse(Arrays.asList(loadBalancer.getCostFunctionNames()) + .contains(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(PrefetchCacheCostFunction.class.getSimpleName())); } @Test - public void testPrefetchCost() throws Exception { + public void testPrefetchCost() { conf.set(HConstants.PREFETCH_PERSISTENCE_PATH_KEY, "/tmp/prefetch.persistence"); CostFunction costFunction = new PrefetchCacheCostFunction(conf); @@ -211,14 +236,13 @@ public void testPrefetchCost() throws Exception { } private class MockClusterForPrefetch extends BalancerClusterState { - private int[][] regionServerPrefetch = null; // [region][server] = prefetch percent + private final int[][] regionServerPrefetch; // [region][server] = prefetch percent public MockClusterForPrefetch(int[][] regionsArray) { // regions[0] is an array where index = serverIndex and value = number of regions super(mockClusterServers(regionsArray[0], 1), null, null, null, null); regionServerPrefetch = new int[regionsArray.length - 1][]; - Map> historicalPrefetchRatio = - new HashMap>(); + Map> historicalPrefetchRatio = new HashMap<>(); for (int i = 1; i < regionsArray.length; i++) { int regionIndex = i - 1; regionServerPrefetch[regionIndex] = new int[regionsArray[i].length - 1]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java index dafb5b965827..68a3f36d56a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestPrefetchCacheCostBalancer.java @@ -22,9 +22,10 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Map; +import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -41,11 +42,9 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.io.ByteBuffAllocator; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.After; import org.junit.Before; import org.junit.ClassRule; @@ -66,7 +65,7 @@ public class TestPrefetchCacheCostBalancer extends StochasticBalancerTestBase { private static final int REGION_SERVERS = 3; - private static final int REGION_NUM = REGION_SERVERS * 3; + private static final int REGION_NUM = REGION_SERVERS * 50; private Admin admin; @@ -154,6 +153,7 @@ public void testOnlyPrefetchCacheCostFunctionDisabled() throws Exception { // Disable the prefetch cache cost function conf.setFloat("hbase.master.balancer.stochastic.prefetchCacheCost", 0.0f); + loadBalancer.loadConf(conf); TEST_UTIL.startMiniCluster(REGION_SERVERS); TEST_UTIL.getDFSCluster().waitClusterUp(); @@ -168,7 +168,7 @@ public void testOnlyPrefetchCacheCostFunctionDisabled() throws Exception { TEST_UTIL.waitTableAvailable(tableName); TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); admin.flush(tableName); - compactTable(tableName); + TEST_UTIL.compact(true); // Validate that all the other cost functions are enabled Arrays.stream(FunctionCostKeys.values()) @@ -189,7 +189,7 @@ public void testOnlyPrefetchCacheCostFunctionDisabled() throws Exception { assertEquals(REGION_SERVERS, ssmap.size()); // Get the name of the region server to shutdown and restart - ServerName serverName = cluster.getRegionServer(REGION_SERVERS - 1).getServerName(); + ServerName serverName = cluster.getClusterMetrics().getServersName().get(REGION_SERVERS - 1); ServerMetrics sm = ssmap.get(serverName); // Verify that some regions are assigned to this region server assertTrue(0.0f != sm.getRegionMetrics().size()); @@ -199,10 +199,10 @@ public void testOnlyPrefetchCacheCostFunctionDisabled() throws Exception { cluster.stopRegionServer(serverName); cluster.waitForRegionServerToStop(serverName, 1000); // Compact the table so that all the regions are reassigned to the running region servers - compactTable(tableName); + TEST_UTIL.compact(true); TEST_UTIL.waitUntilNoRegionsInTransition(12000); - ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + ssmap = cluster.getClusterMetrics().getLiveServerMetrics(); assertEquals(REGION_SERVERS - 1, ssmap.size()); sm = ssmap.get(serverName); // Validate that no server metrics is found for the non-active server @@ -215,11 +215,13 @@ public void testOnlyPrefetchCacheCostFunctionDisabled() throws Exception { cluster.waitForRegionServerToStart(serverName.getHostname(), serverName.getPort(), 1000); admin.balance(); TEST_UTIL.waitUntilNoRegionsInTransition(12000); - ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + ssmap = cluster.getClusterMetrics().getLiveServerMetrics(); assertEquals(REGION_SERVERS, ssmap.size()); - serverName = cluster.getRegionServer(REGION_SERVERS - 1).getServerName(); - sm = ssmap.get(serverName); + ServerName newServerName = cluster.getClusterMetrics().getServersName().get(REGION_SERVERS - 1); + // Verify that the same region server has been started + assertTrue(ServerName.isSameAddress(serverName, newServerName)); + sm = ssmap.get(newServerName); assertNotNull(sm); assertTrue(sm.getRegionMetrics().size() > 0); @@ -243,6 +245,7 @@ public void testOnlyPrefetchCacheCostFunctionEnabled() throws Exception { Arrays.stream(FunctionCostKeys.values()) .forEach(functionCostKey -> conf.setFloat(functionCostKey.getValue(), 0.0f)); + loadBalancer.loadConf(conf); TEST_UTIL.startMiniCluster(REGION_SERVERS); TEST_UTIL.getDFSCluster().waitClusterUp(); @@ -256,7 +259,7 @@ public void testOnlyPrefetchCacheCostFunctionEnabled() throws Exception { TEST_UTIL.waitTableAvailable(tableName); TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); admin.flush(tableName); - compactTable(tableName); + TEST_UTIL.compact(true); // Validate that all the other cost functions are disabled Arrays.stream(FunctionCostKeys.values()) @@ -274,6 +277,7 @@ public void testOnlyPrefetchCacheCostFunctionEnabled() throws Exception { TEST_UTIL.waitUntilNoRegionsInTransition(120000); Map ssmap = cluster.getClusterMetrics().getLiveServerMetrics(); + assertEquals(REGION_SERVERS, ssmap.size()); // Shutdown the last server. This is because the server id for an inactive server is reassigned @@ -281,15 +285,15 @@ public void testOnlyPrefetchCacheCostFunctionEnabled() throws Exception { // available // server id. In our case, we want to track the same server and hence, it's safe to restart the // last server in the list - ServerName serverName = cluster.getRegionServer(REGION_SERVERS - 1).getServerName(); + ServerName serverName = cluster.getClusterMetrics().getServersName().get(REGION_SERVERS - 1); ServerMetrics sm = ssmap.get(serverName); assertTrue(0 != sm.getRegionMetrics().size()); cluster.stopRegionServer(serverName); cluster.waitForRegionServerToStop(serverName, 1000); - compactTable(tableName); + TEST_UTIL.compact(true); TEST_UTIL.waitUntilNoRegionsInTransition(12000); - ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + ssmap = cluster.getClusterMetrics().getLiveServerMetrics(); assertEquals(REGION_SERVERS - 1, ssmap.size()); sm = ssmap.get(serverName); assertNull(sm); @@ -299,21 +303,91 @@ public void testOnlyPrefetchCacheCostFunctionEnabled() throws Exception { cluster.waitForRegionServerToStart(serverName.getHostname(), serverName.getPort(), 1000); admin.balance(); TEST_UTIL.waitUntilNoRegionsInTransition(120000); - ssmap = admin.getClusterMetrics().getLiveServerMetrics(); + ssmap = cluster.getClusterMetrics().getLiveServerMetrics(); assertEquals(REGION_SERVERS, ssmap.size()); - serverName = cluster.getRegionServer(REGION_SERVERS - 1).getServerName(); - sm = ssmap.get(serverName); + ServerName newServerName = cluster.getClusterMetrics().getServersName().get(REGION_SERVERS - 1); + // Verify that the same region server has been started + assertTrue(ServerName.isSameAddress(serverName, newServerName)); + sm = ssmap.get(newServerName); assertNotNull(sm); assertEquals(0, sm.getRegionMetrics().size()); } - private void compactTable(TableName tableName) throws IOException { - for (JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) { - for (HRegion region : t.getRegionServer().getRegions(tableName)) { - region.compact(true); - region.flush(true); + @Test + public void testStressTestWithOnlyPrefetchCacheCostFunctionEnabled() throws Exception { + // Test the prefetch cache cost returned by the cost function when random servers are + // restarted and only the PrefetchCacheCostFunction is enabled. Ensure that the prefetch cost + // returned by the cost function is always between 0 and 1. + + // Disable all other cost functions + Arrays.stream(FunctionCostKeys.values()) + .forEach(functionCostKey -> conf.setFloat(functionCostKey.getValue(), 0.0f)); + loadBalancer.loadConf(conf); + + TEST_UTIL.startMiniCluster(REGION_SERVERS); + TEST_UTIL.getDFSCluster().waitClusterUp(); + cluster = TEST_UTIL.getHBaseCluster(); + admin = TEST_UTIL.getAdmin(); + admin.balancerSwitch(false, true); + TableName tableName = TableName.valueOf("testTableOnlyPrefetchCacheCostFunctionEnabled"); + TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build(); + admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TEST_UTIL.waitTableAvailable(tableName); + TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); + admin.flush(tableName); + TEST_UTIL.compact(true); + + // Verify that all the other cost functions except the PrefetchCacheCostFunction are disabled + Arrays.stream(FunctionCostKeys.values()) + .forEach(functionCostKey -> verifyCostFunctionState(admin.getConfiguration(), + functionCostKey.getValue(), false)); + + verifyCostFunctionState(admin.getConfiguration(), + "hbase.master.balancer.stochastic.prefetchCacheCost", true); + + admin.balancerSwitch(true, true); + admin.balance(); + TEST_UTIL.waitUntilNoRegionsInTransition(120000); + + Random rand = new Random(); + for (int i = 0; i < 5; i++) { + int randomServerID = rand.nextInt(REGION_SERVERS); + ServerName sn = cluster.getClusterMetrics().getServersName().get(randomServerID); + cluster.stopRegionServer(sn); + cluster.waitForRegionServerToStop(sn, 1000); + TEST_UTIL.compact(true); + TEST_UTIL.waitUntilNoRegionsInTransition(12000); + cluster.startRegionServer(sn.getHostname(), sn.getPort()); + cluster.waitForRegionServerToStart(sn.getHostname(), sn.getPort(), 1000); + admin.balance(); + // Verify that the same server was restarted + verifyServerActive(sn); + assertEquals(REGION_SERVERS, cluster.getClusterMetrics().getLiveServerMetrics().size()); + validatePrefetchCacheCost(loadBalancer.getCostFunctions()); + } + } + + private void verifyServerActive(ServerName serverName) throws Exception { + // The server id of the region server may change post restart. The only way to ensure that the + // same server has been restarted is by searching for the server address (host:port) in the + // active server list + boolean found = false; + for (ServerName sname : cluster.getClusterMetrics().getServersName()) { + if (ServerName.isSameAddress(sname, serverName)) { + found = true; + break; + } + } + assertTrue(found); + } + + private void validatePrefetchCacheCost(List cf) { + for (CostFunction c : cf) { + if (c.getMultiplier() > 0.0f) { + assertTrue(c.cost() >= 0.0 && c.cost() <= 1.0); } } }