Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id D448D200D5C for ; Fri, 15 Dec 2017 17:54:03 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id D2940160C14; Fri, 15 Dec 2017 16:54:03 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id F1B6C160C04 for ; Fri, 15 Dec 2017 17:54:02 +0100 (CET) Received: (qmail 75182 invoked by uid 500); 15 Dec 2017 16:54:02 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 75172 invoked by uid 99); 15 Dec 2017 16:54:02 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 15 Dec 2017 16:54:02 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 032D5DFF75; Fri, 15 Dec 2017 16:54:02 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: stack@apache.org To: commits@hbase.apache.org Date: Fri, 15 Dec 2017 16:54:02 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/2] hbase git commit: HBASE-18946 Stochastic load balancer assigns replica regions to the same RS archived-at: Fri, 15 Dec 2017 16:54:04 -0000 Repository: hbase Updated Branches: refs/heads/master 11e82de8a -> 010012cbc http://git-wip-us.apache.org/repos/asf/hbase/blob/010012cb/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java new file mode 100644 index 0000000..1646a66 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java @@ -0,0 +1,164 @@ +/* + * 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.regionserver; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionReplicaUtil; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.RegionSplitter; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.junit.rules.TestRule; + +@Category({RegionServerTests.class, MediumTests.class}) +public class TestRegionReplicasWithRestartScenarios { + private static final Log LOG = LogFactory.getLog(TestRegionReplicasWithRestartScenarios.class); + @Rule public TestName name = new TestName(); + @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). + withTimeout(this.getClass()). + withLookingForStuckThread(true). + build(); + + private static final int NB_SERVERS = 3; + private Table table; + + private static final HBaseTestingUtility HTU = new HBaseTestingUtility(); + private static final byte[] f = HConstants.CATALOG_FAMILY; + + @BeforeClass + public static void beforeClass() throws Exception { + // Reduce the hdfs block size and prefetch to trigger the file-link reopen + // when the file is moved to archive (e.g. compaction) + HTU.getConfiguration().setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 8192); + HTU.getConfiguration().setInt(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, 1); + HTU.getConfiguration().setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 128 * 1024 * 1024); + HTU.getConfiguration().setInt(">hbase.master.wait.on.regionservers.mintostart", 3); + HTU.startMiniCluster(NB_SERVERS); + } + + @Before + public void before() throws IOException { + TableName tableName = TableName.valueOf(this.name.getMethodName()); + // Create table then get the single region for our new table. + this.table = createTableDirectlyFromHTD(tableName); + } + + @After + public void after() throws IOException { + this.table.close(); + } + + private static Table createTableDirectlyFromHTD(final TableName tableName) throws IOException { + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName); + builder.setRegionReplication(3); + return HTU.createTable(builder.build(), new byte[][] { f }, getSplits(20), + new Configuration(HTU.getConfiguration())); + } + + private static byte[][] getSplits(int numRegions) { + RegionSplitter.UniformSplit split = new RegionSplitter.UniformSplit(); + split.setFirstRow(Bytes.toBytes(0L)); + split.setLastRow(Bytes.toBytes(Long.MAX_VALUE)); + return split.split(numRegions); + } + + @AfterClass + public static void afterClass() throws Exception { + HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false; + HTU.shutdownMiniCluster(); + } + + private HRegionServer getRS() { + return HTU.getMiniHBaseCluster().getRegionServer(0); + } + + private HRegionServer getSecondaryRS() { + return HTU.getMiniHBaseCluster().getRegionServer(1); + } + + private HRegionServer getTertiaryRS() { + return HTU.getMiniHBaseCluster().getRegionServer(2); + } + + @Test + public void testRegionReplicasCreated() throws Exception { + Collection onlineRegions = getRS().getOnlineRegionsLocalContext(); + boolean res = checkDuplicates(onlineRegions); + assertFalse(res); + Collection onlineRegions2 = getSecondaryRS().getOnlineRegionsLocalContext(); + res = checkDuplicates(onlineRegions2); + assertFalse(res); + Collection onlineRegions3 = getTertiaryRS().getOnlineRegionsLocalContext(); + checkDuplicates(onlineRegions3); + assertFalse(res); + int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); + assertEquals(62, totalRegions); + } + + private boolean checkDuplicates(Collection onlineRegions3) throws Exception { + ArrayList copyOfRegion = new ArrayList(onlineRegions3); + for (Region region : copyOfRegion) { + RegionInfo regionInfo = region.getRegionInfo(); + RegionInfo regionInfoForReplica = + RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo); + int i = 0; + for (Region actualRegion : onlineRegions3) { + if (regionInfoForReplica.equals( + RegionReplicaUtil.getRegionInfoForDefaultReplica(actualRegion.getRegionInfo()))) { + i++; + if (i > 1) { + LOG.info("Duplicate found " + actualRegion.getRegionInfo() + " " + + region.getRegionInfo()); + assertTrue(Bytes.equals(region.getRegionInfo().getStartKey(), + actualRegion.getRegionInfo().getStartKey())); + assertTrue(Bytes.equals(region.getRegionInfo().getEndKey(), + actualRegion.getRegionInfo().getEndKey())); + return true; + } + } + } + } + return false; + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/010012cb/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index 8924454..92833fd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -716,7 +716,7 @@ public class TestSplitTransactionOnCluster { assertTrue(regionStates.isRegionInState(daughters.get(1).getRegionInfo(), State.OPEN)); // We should not be able to assign it again - am.assign(hri, true); + am.assign(hri); assertFalse("Split region can't be assigned", regionStates.isRegionInTransition(hri)); assertTrue(regionStates.isRegionInState(hri, State.SPLIT));