HBASE-19867 Split TestStochasticLoadBalancer into several small tests

This commit is contained in:
zhangduo 2018-01-27 08:03:41 +08:00
parent a5a8c4f3f2
commit f1502a3aca
9 changed files with 545 additions and 325 deletions

View File

@ -34,8 +34,11 @@ import java.util.Set;
import java.util.SortedSet;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.ThreadLocalRandom;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
@ -48,6 +51,8 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.net.DNSToSwitchMapping;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.rules.TestRule;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -59,11 +64,13 @@ import org.slf4j.LoggerFactory;
*/
public class BalancerTestBase {
private static final Logger LOG = LoggerFactory.getLogger(BalancerTestBase.class);
protected static Random rand = new Random();
static int regionId = 0;
protected static Configuration conf;
protected static StochasticLoadBalancer loadBalancer;
@Rule
public final TestRule timeout = CategoryBasedTimeout.forClass(getClass());
@BeforeClass
public static void beforeAllTests() throws Exception {
conf = HBaseConfiguration.create();
@ -161,11 +168,7 @@ public class BalancerTestBase {
@Override
public List<String> resolve(List<String> names) {
List<String> ret = new ArrayList<>(names.size());
for (String name : names) {
ret.add("rack");
}
return ret;
return Stream.generate(() -> "rack").limit(names.size()).collect(Collectors.toList());
}
// do not add @Override annotations here. It mighty break compilation with earlier Hadoops
@ -438,6 +441,7 @@ public class BalancerTestBase {
List<RegionInfo> regions = new ArrayList<>(numRegions);
byte[] start = new byte[16];
byte[] end = new byte[16];
Random rand = ThreadLocalRandom.current();
rand.nextBytes(start);
rand.nextBytes(end);
for (int i = 0; i < numRegions; i++) {
@ -464,6 +468,7 @@ public class BalancerTestBase {
List<RegionInfo> regions = new ArrayList<>(numRegions);
byte[] start = new byte[16];
byte[] end = new byte[16];
Random rand = ThreadLocalRandom.current();
rand.nextBytes(start);
rand.nextBytes(end);
for (int i = 0; i < numRegions; i++) {
@ -492,6 +497,7 @@ public class BalancerTestBase {
ServerName sn = this.serverQueue.poll();
return new ServerAndLoad(sn, numRegionsPerServer);
}
Random rand = ThreadLocalRandom.current();
String host = "srv" + rand.nextInt(Integer.MAX_VALUE);
int port = rand.nextInt(60000);
long startCode = rand.nextLong();

View File

@ -25,11 +25,8 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Queue;
import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
@ -40,25 +37,19 @@ import org.apache.hadoop.hbase.ServerMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Size;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.MockNoopMasterServices;
import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer.ServerLocalityCostFunction;
import org.apache.hadoop.hbase.testclassification.FlakeyTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Category({FlakeyTests.class, MediumTests.class})
@Category({ MasterTests.class, MediumTests.class })
public class TestStochasticLoadBalancer extends BalancerTestBase {
public static final String REGION_KEY = "testRegion";
private static final Logger LOG = LoggerFactory.getLogger(TestStochasticLoadBalancer.class);
private static final String REGION_KEY = "testRegion";
// Mapping of locality test -> expected locality
private float[] expectedLocalities = {1.0f, 0.0f, 0.50f, 0.25f, 1.0f};
@ -118,7 +109,6 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
},
};
@Test
public void testKeepRegionLoad() throws Exception {
@ -172,37 +162,6 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
loadBalancer.setConf(conf);
}
/**
* Test the load balancing algorithm.
*
* Invariant is that all servers should be hosting either floor(average) or
* ceiling(average)
*
* @throws Exception
*/
@Test
public void testBalanceCluster() throws Exception {
conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
loadBalancer.setConf(conf);
for (int[] mockCluster : clusterStateMocks) {
Map<ServerName, List<RegionInfo>> servers = mockClusterServers(mockCluster);
List<ServerAndLoad> list = convertToList(servers);
LOG.info("Mock Cluster : " + printMock(list) + " " + printStats(list));
List<RegionPlan> plans = loadBalancer.balanceCluster(servers);
List<ServerAndLoad> balancedCluster = reconcile(list, plans, servers);
LOG.info("Mock Balance : " + printMock(balancedCluster));
assertClusterAsBalanced(balancedCluster);
List<RegionPlan> secondPlans = loadBalancer.balanceCluster(servers);
assertNull(secondPlans);
for (Map.Entry<ServerName, List<RegionInfo>> entry : servers.entrySet()) {
returnRegions(entry.getValue());
returnServer(entry.getKey());
}
}
}
@Test
public void testLocalityCost() throws Exception {
Configuration conf = HBaseConfiguration.create();
@ -371,7 +330,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
assertEquals(0.5, costFunction.costFromArray(statThree), 0.01);
}
@Test(timeout = 60000)
@Test
public void testLosingRs() throws Exception {
int numNodes = 3;
int numRegions = 20;
@ -402,277 +361,6 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
assertNull(plans);
}
@Test
public void testReplicaCost() {
Configuration conf = HBaseConfiguration.create();
StochasticLoadBalancer.CostFunction
costFunction = new StochasticLoadBalancer.RegionReplicaHostCostFunction(conf);
for (int[] mockCluster : clusterStateMocks) {
BaseLoadBalancer.Cluster cluster = mockCluster(mockCluster);
costFunction.init(cluster);
double cost = costFunction.cost();
assertTrue(cost >= 0);
assertTrue(cost <= 1.01);
}
}
@Test
public void testReplicaCostForReplicas() {
Configuration conf = HBaseConfiguration.create();
StochasticLoadBalancer.CostFunction
costFunction = new StochasticLoadBalancer.RegionReplicaHostCostFunction(conf);
int [] servers = new int[] {3,3,3,3,3};
TreeMap<ServerName, List<RegionInfo>> clusterState = mockClusterServers(servers);
BaseLoadBalancer.Cluster cluster;
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
costFunction.init(cluster);
double costWithoutReplicas = costFunction.cost();
assertEquals(0, costWithoutReplicas, 0);
// replicate the region from first server to the last server
RegionInfo replica1 = RegionReplicaUtil.getRegionInfoForReplica(
clusterState.firstEntry().getValue().get(0),1);
clusterState.lastEntry().getValue().add(replica1);
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
costFunction.init(cluster);
double costWith1ReplicaDifferentServer = costFunction.cost();
assertEquals(0, costWith1ReplicaDifferentServer, 0);
// add a third replica to the last server
RegionInfo replica2 = RegionReplicaUtil.getRegionInfoForReplica(replica1, 2);
clusterState.lastEntry().getValue().add(replica2);
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
costFunction.init(cluster);
double costWith1ReplicaSameServer = costFunction.cost();
assertTrue(costWith1ReplicaDifferentServer < costWith1ReplicaSameServer);
// test with replication = 4 for following:
RegionInfo replica3;
Iterator<Entry<ServerName, List<RegionInfo>>> it;
Entry<ServerName, List<RegionInfo>> entry;
clusterState = mockClusterServers(servers);
it = clusterState.entrySet().iterator();
entry = it.next(); //first server
RegionInfo hri = entry.getValue().get(0);
replica1 = RegionReplicaUtil.getRegionInfoForReplica(hri, 1);
replica2 = RegionReplicaUtil.getRegionInfoForReplica(hri, 2);
replica3 = RegionReplicaUtil.getRegionInfoForReplica(hri, 3);
entry.getValue().add(replica1);
entry.getValue().add(replica2);
it.next().getValue().add(replica3); //2nd server
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
costFunction.init(cluster);
double costWith3ReplicasSameServer = costFunction.cost();
clusterState = mockClusterServers(servers);
hri = clusterState.firstEntry().getValue().get(0);
replica1 = RegionReplicaUtil.getRegionInfoForReplica(hri, 1);
replica2 = RegionReplicaUtil.getRegionInfoForReplica(hri, 2);
replica3 = RegionReplicaUtil.getRegionInfoForReplica(hri, 3);
clusterState.firstEntry().getValue().add(replica1);
clusterState.lastEntry().getValue().add(replica2);
clusterState.lastEntry().getValue().add(replica3);
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
costFunction.init(cluster);
double costWith2ReplicasOnTwoServers = costFunction.cost();
assertTrue(costWith2ReplicasOnTwoServers < costWith3ReplicasSameServer);
}
@Test
public void testNeedsBalanceForColocatedReplicas() {
// check for the case where there are two hosts and with one rack, and where
// both the replicas are hosted on the same server
List<RegionInfo> regions = randomRegions(1);
ServerName s1 = ServerName.valueOf("host1", 1000, 11111);
ServerName s2 = ServerName.valueOf("host11", 1000, 11111);
Map<ServerName, List<RegionInfo>> map = new HashMap<>();
map.put(s1, regions);
regions.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1));
// until the step above s1 holds two replicas of a region
regions = randomRegions(1);
map.put(s2, regions);
assertTrue(loadBalancer.needsBalance(new Cluster(map, null, null, null)));
// check for the case where there are two hosts on the same rack and there are two racks
// and both the replicas are on the same rack
map.clear();
regions = randomRegions(1);
List<RegionInfo> regionsOnS2 = new ArrayList<>(1);
regionsOnS2.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1));
map.put(s1, regions);
map.put(s2, regionsOnS2);
// add another server so that the cluster has some host on another rack
map.put(ServerName.valueOf("host2", 1000, 11111), randomRegions(1));
assertTrue(loadBalancer.needsBalance(new Cluster(map, null, null,
new ForTestRackManagerOne())));
}
@Test (timeout = 60000)
public void testSmallCluster() {
int numNodes = 10;
int numRegions = 1000;
int numRegionsPerServer = 40; //all servers except one
int replication = 1;
int numTables = 10;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
}
@Test (timeout = 60000)
public void testSmallCluster2() {
int numNodes = 20;
int numRegions = 2000;
int numRegionsPerServer = 40; //all servers except one
int replication = 1;
int numTables = 10;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
}
@Test (timeout = 60000)
public void testSmallCluster3() {
int numNodes = 20;
int numRegions = 2000;
int numRegionsPerServer = 1; // all servers except one
int replication = 1;
int numTables = 10;
/* fails because of max moves */
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, false, false);
}
@Test (timeout = 800000)
public void testMidCluster() {
int numNodes = 100;
int numRegions = 10000;
int numRegionsPerServer = 60; // all servers except one
int replication = 1;
int numTables = 40;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
}
@Test (timeout = 800000)
public void testMidCluster2() {
int numNodes = 200;
int numRegions = 100000;
int numRegionsPerServer = 40; // all servers except one
int replication = 1;
int numTables = 400;
testWithCluster(numNodes,
numRegions,
numRegionsPerServer,
replication,
numTables,
false, /* num large num regions means may not always get to best balance with one run */
false);
}
@Test (timeout = 800000)
public void testMidCluster3() {
int numNodes = 100;
int numRegions = 2000;
int numRegionsPerServer = 9; // all servers except one
int replication = 1;
int numTables = 110;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
// TODO(eclark): Make sure that the tables are well distributed.
}
@Test
public void testLargeCluster() {
int numNodes = 1000;
int numRegions = 100000; //100 regions per RS
int numRegionsPerServer = 80; //all servers except one
int numTables = 100;
int replication = 1;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
}
@Test (timeout = 800000)
public void testRegionReplicasOnSmallCluster() {
int numNodes = 10;
int numRegions = 1000;
int replication = 3; // 3 replicas per region
int numRegionsPerServer = 80; //all regions are mostly balanced
int numTables = 10;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
}
@Ignore @Test (timeout = 800000) // Test is flakey. TODO: Fix!
public void testRegionReplicationOnMidClusterSameHosts() {
conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
loadBalancer.setConf(conf);
int numHosts = 100;
int numRegions = 100 * 100;
int replication = 3; // 3 replicas per region
int numRegionsPerServer = 5;
int numTables = 10;
Map<ServerName, List<RegionInfo>> serverMap =
createServerMap(numHosts, numRegions, numRegionsPerServer, replication, numTables);
int numNodesPerHost = 4;
// create a new map with 4 RS per host.
Map<ServerName, List<RegionInfo>> newServerMap = new TreeMap<>(serverMap);
for (Map.Entry<ServerName, List<RegionInfo>> entry : serverMap.entrySet()) {
for (int i=1; i < numNodesPerHost; i++) {
ServerName s1 = entry.getKey();
ServerName s2 = ServerName.valueOf(s1.getHostname(), s1.getPort() + i, 1); // create an RS for the same host
newServerMap.put(s2, new ArrayList<>());
}
}
testWithCluster(newServerMap, null, true, true);
}
private static class ForTestRackManager extends RackManager {
int numRacks;
public ForTestRackManager(int numRacks) {
this.numRacks = numRacks;
}
@Override
public String getRack(ServerName server) {
return "rack_" + (server.hashCode() % numRacks);
}
}
private static class ForTestRackManagerOne extends RackManager {
@Override
public String getRack(ServerName server) {
return server.getHostname().endsWith("1") ? "rack1" : "rack2";
}
}
@Test (timeout = 800000)
public void testRegionReplicationOnMidClusterWithRacks() {
conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 10000000L);
conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
loadBalancer.setConf(conf);
int numNodes = 30;
int numRegions = numNodes * 30;
int replication = 3; // 3 replicas per region
int numRegionsPerServer = 28;
int numTables = 10;
int numRacks = 4; // all replicas should be on a different rack
Map<ServerName, List<RegionInfo>> serverMap =
createServerMap(numNodes, numRegions, numRegionsPerServer, replication, numTables);
RackManager rm = new ForTestRackManager(numRacks);
testWithCluster(serverMap, rm, false, true);
}
// This mock allows us to test the LocalityCostFunction
private class MockCluster extends BaseLoadBalancer.Cluster {
@ -705,7 +393,5 @@ public class TestStochasticLoadBalancer extends BalancerTestBase {
public int getRegionSizeMB(int region) {
return 1;
}
}
}

View File

@ -0,0 +1,67 @@
/**
* 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.assertNull;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Category({ MasterTests.class, LargeTests.class })
public class TestStochasticLoadBalancerBalanceCluster extends BalancerTestBase {
private static final Logger LOG =
LoggerFactory.getLogger(TestStochasticLoadBalancerBalanceCluster.class);
/**
* Test the load balancing algorithm.
* <p>
* Invariant is that all servers should be hosting either floor(average) or ceiling(average)
*/
@Test
public void testBalanceCluster() throws Exception {
conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
loadBalancer.setConf(conf);
for (int[] mockCluster : clusterStateMocks) {
Map<ServerName, List<RegionInfo>> servers = mockClusterServers(mockCluster);
List<ServerAndLoad> list = convertToList(servers);
LOG.info("Mock Cluster : " + printMock(list) + " " + printStats(list));
List<RegionPlan> plans = loadBalancer.balanceCluster(servers);
List<ServerAndLoad> balancedCluster = reconcile(list, plans, servers);
LOG.info("Mock Balance : " + printMock(balancedCluster));
assertClusterAsBalanced(balancedCluster);
List<RegionPlan> secondPlans = loadBalancer.balanceCluster(servers);
assertNull(secondPlans);
for (Map.Entry<ServerName, List<RegionInfo>> entry : servers.entrySet()) {
returnRegions(entry.getValue());
returnServer(entry.getKey());
}
}
}
}

View File

@ -0,0 +1,37 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.balancer;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MasterTests.class, MediumTests.class })
public class TestStochasticLoadBalancerLargeCluster extends BalancerTestBase {
@Test
public void testLargeCluster() {
int numNodes = 1000;
int numRegions = 100000; // 100 regions per RS
int numRegionsPerServer = 80; // all servers except one
int numTables = 100;
int replication = 1;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
}
}

View File

@ -0,0 +1,61 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.balancer;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MasterTests.class, LargeTests.class })
public class TestStochasticLoadBalancerMidCluster extends BalancerTestBase {
@Test
public void testMidCluster() {
int numNodes = 100;
int numRegions = 10000;
int numRegionsPerServer = 60; // all servers except one
int replication = 1;
int numTables = 40;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
}
@Test
public void testMidCluster2() {
int numNodes = 200;
int numRegions = 100000;
int numRegionsPerServer = 40; // all servers except one
int replication = 1;
int numTables = 400;
// num large num regions means may not always get to best balance with one run
boolean assertFullyBalanced = false;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables,
assertFullyBalanced, false);
}
@Test
public void testMidCluster3() {
int numNodes = 100;
int numRegions = 2000;
int numRegionsPerServer = 9; // all servers except one
int replication = 1;
int numTables = 110;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
// TODO(eclark): Make sure that the tables are well distributed.
}
}

View File

@ -0,0 +1,178 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.balancer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MasterTests.class, LargeTests.class })
public class TestStochasticLoadBalancerRegionReplica extends BalancerTestBase {
@Test
public void testReplicaCost() {
Configuration conf = HBaseConfiguration.create();
StochasticLoadBalancer.CostFunction costFunction =
new StochasticLoadBalancer.RegionReplicaHostCostFunction(conf);
for (int[] mockCluster : clusterStateMocks) {
BaseLoadBalancer.Cluster cluster = mockCluster(mockCluster);
costFunction.init(cluster);
double cost = costFunction.cost();
assertTrue(cost >= 0);
assertTrue(cost <= 1.01);
}
}
@Test
public void testReplicaCostForReplicas() {
Configuration conf = HBaseConfiguration.create();
StochasticLoadBalancer.CostFunction costFunction =
new StochasticLoadBalancer.RegionReplicaHostCostFunction(conf);
int[] servers = new int[] { 3, 3, 3, 3, 3 };
TreeMap<ServerName, List<RegionInfo>> clusterState = mockClusterServers(servers);
BaseLoadBalancer.Cluster cluster;
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
costFunction.init(cluster);
double costWithoutReplicas = costFunction.cost();
assertEquals(0, costWithoutReplicas, 0);
// replicate the region from first server to the last server
RegionInfo replica1 =
RegionReplicaUtil.getRegionInfoForReplica(clusterState.firstEntry().getValue().get(0), 1);
clusterState.lastEntry().getValue().add(replica1);
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
costFunction.init(cluster);
double costWith1ReplicaDifferentServer = costFunction.cost();
assertEquals(0, costWith1ReplicaDifferentServer, 0);
// add a third replica to the last server
RegionInfo replica2 = RegionReplicaUtil.getRegionInfoForReplica(replica1, 2);
clusterState.lastEntry().getValue().add(replica2);
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
costFunction.init(cluster);
double costWith1ReplicaSameServer = costFunction.cost();
assertTrue(costWith1ReplicaDifferentServer < costWith1ReplicaSameServer);
// test with replication = 4 for following:
RegionInfo replica3;
Iterator<Entry<ServerName, List<RegionInfo>>> it;
Entry<ServerName, List<RegionInfo>> entry;
clusterState = mockClusterServers(servers);
it = clusterState.entrySet().iterator();
entry = it.next(); // first server
RegionInfo hri = entry.getValue().get(0);
replica1 = RegionReplicaUtil.getRegionInfoForReplica(hri, 1);
replica2 = RegionReplicaUtil.getRegionInfoForReplica(hri, 2);
replica3 = RegionReplicaUtil.getRegionInfoForReplica(hri, 3);
entry.getValue().add(replica1);
entry.getValue().add(replica2);
it.next().getValue().add(replica3); // 2nd server
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
costFunction.init(cluster);
double costWith3ReplicasSameServer = costFunction.cost();
clusterState = mockClusterServers(servers);
hri = clusterState.firstEntry().getValue().get(0);
replica1 = RegionReplicaUtil.getRegionInfoForReplica(hri, 1);
replica2 = RegionReplicaUtil.getRegionInfoForReplica(hri, 2);
replica3 = RegionReplicaUtil.getRegionInfoForReplica(hri, 3);
clusterState.firstEntry().getValue().add(replica1);
clusterState.lastEntry().getValue().add(replica2);
clusterState.lastEntry().getValue().add(replica3);
cluster = new BaseLoadBalancer.Cluster(clusterState, null, null, null);
costFunction.init(cluster);
double costWith2ReplicasOnTwoServers = costFunction.cost();
assertTrue(costWith2ReplicasOnTwoServers < costWith3ReplicasSameServer);
}
@Test
public void testNeedsBalanceForColocatedReplicas() {
// check for the case where there are two hosts and with one rack, and where
// both the replicas are hosted on the same server
List<RegionInfo> regions = randomRegions(1);
ServerName s1 = ServerName.valueOf("host1", 1000, 11111);
ServerName s2 = ServerName.valueOf("host11", 1000, 11111);
Map<ServerName, List<RegionInfo>> map = new HashMap<>();
map.put(s1, regions);
regions.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1));
// until the step above s1 holds two replicas of a region
regions = randomRegions(1);
map.put(s2, regions);
assertTrue(loadBalancer.needsBalance(new Cluster(map, null, null, null)));
// check for the case where there are two hosts on the same rack and there are two racks
// and both the replicas are on the same rack
map.clear();
regions = randomRegions(1);
List<RegionInfo> regionsOnS2 = new ArrayList<>(1);
regionsOnS2.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(0), 1));
map.put(s1, regions);
map.put(s2, regionsOnS2);
// add another server so that the cluster has some host on another rack
map.put(ServerName.valueOf("host2", 1000, 11111), randomRegions(1));
assertTrue(
loadBalancer.needsBalance(new Cluster(map, null, null, new ForTestRackManagerOne())));
}
@Test
public void testRegionReplicasOnSmallCluster() {
int numNodes = 10;
int numRegions = 1000;
int replication = 3; // 3 replicas per region
int numRegionsPerServer = 80; // all regions are mostly balanced
int numTables = 10;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
}
private static class ForTestRackManagerOne extends RackManager {
@Override
public String getRack(ServerName server) {
return server.getHostname().endsWith("1") ? "rack1" : "rack2";
}
}
}

View File

@ -0,0 +1,62 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.balancer;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MasterTests.class, LargeTests.class })
public class TestStochasticLoadBalancerRegionReplicaSameHosts extends BalancerTestBase {
@Test // Test is flakey. TODO: Fix!
public void testRegionReplicationOnMidClusterSameHosts() {
conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
loadBalancer.setConf(conf);
int numHosts = 100;
int numRegions = 100 * 100;
int replication = 3; // 3 replicas per region
int numRegionsPerServer = 5;
int numTables = 10;
Map<ServerName, List<RegionInfo>> serverMap =
createServerMap(numHosts, numRegions, numRegionsPerServer, replication, numTables);
int numNodesPerHost = 4;
// create a new map with 4 RS per host.
Map<ServerName, List<RegionInfo>> newServerMap = new TreeMap<>(serverMap);
for (Map.Entry<ServerName, List<RegionInfo>> entry : serverMap.entrySet()) {
for (int i = 1; i < numNodesPerHost; i++) {
ServerName s1 = entry.getKey();
// create an RS for the same host
ServerName s2 = ServerName.valueOf(s1.getHostname(), s1.getPort() + i, 1);
newServerMap.put(s2, new ArrayList<>());
}
}
testWithCluster(newServerMap, null, true, true);
}
}

View File

@ -0,0 +1,64 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.balancer;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MasterTests.class, LargeTests.class })
public class TestStochasticLoadBalancerRegionReplicaWithRacks extends BalancerTestBase {
private static class ForTestRackManager extends RackManager {
int numRacks;
public ForTestRackManager(int numRacks) {
this.numRacks = numRacks;
}
@Override
public String getRack(ServerName server) {
return "rack_" + (server.hashCode() % numRacks);
}
}
@Test
public void testRegionReplicationOnMidClusterWithRacks() {
conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 10000000L);
conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
loadBalancer.setConf(conf);
int numNodes = 30;
int numRegions = numNodes * 30;
int replication = 3; // 3 replicas per region
int numRegionsPerServer = 28;
int numTables = 10;
int numRacks = 4; // all replicas should be on a different rack
Map<ServerName, List<RegionInfo>> serverMap =
createServerMap(numNodes, numRegions, numRegionsPerServer, replication, numTables);
RackManager rm = new ForTestRackManager(numRacks);
testWithCluster(serverMap, rm, false, true);
}
}

View File

@ -0,0 +1,59 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.balancer;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MasterTests.class, MediumTests.class })
public class TestStochasticLoadBalancerSmallCluster extends BalancerTestBase {
@Test
public void testSmallCluster() {
int numNodes = 10;
int numRegions = 1000;
int numRegionsPerServer = 40; // all servers except one
int replication = 1;
int numTables = 10;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
}
@Test
public void testSmallCluster2() {
int numNodes = 20;
int numRegions = 2000;
int numRegionsPerServer = 40; // all servers except one
int replication = 1;
int numTables = 10;
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
}
@Test
public void testSmallCluster3() {
int numNodes = 20;
int numRegions = 2000;
int numRegionsPerServer = 1; // all servers except one
int replication = 1;
int numTables = 10;
/* fails because of max moves */
testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, false,
false);
}
}