HBASE-22121 AsyncAdmin can not deal with non default meta replica

Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
zhangduo 2019-03-30 18:40:41 +08:00
parent 833736f28a
commit 7fe31b060f
5 changed files with 189 additions and 24 deletions

View File

@ -2301,7 +2301,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
* Get the region location for the passed region name. The region name may be a full region name
* or encoded region name. If the region does not found, then it'll throw an
* UnknownRegionException wrapped by a {@link CompletableFuture}
* @param regionNameOrEncodedRegionName
* @param regionNameOrEncodedRegionName region name or encoded region name
* @return region location, wrapped by a {@link CompletableFuture}
*/
@VisibleForTesting
@ -2312,10 +2312,28 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
try {
CompletableFuture<Optional<HRegionLocation>> future;
if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
future = AsyncMetaTableAccessor.getRegionLocationWithEncodedName(metaTable,
regionNameOrEncodedRegionName);
String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
// old format encodedName, should be meta region
future = connection.registry.getMetaRegionLocation()
.thenApply(locs -> Stream.of(locs.getRegionLocations())
.filter(loc -> loc.getRegion().getEncodedName().equals(encodedName)).findFirst());
} else {
future = AsyncMetaTableAccessor.getRegionLocationWithEncodedName(metaTable,
regionNameOrEncodedRegionName);
}
} else {
future = AsyncMetaTableAccessor.getRegionLocation(metaTable, regionNameOrEncodedRegionName);
RegionInfo regionInfo =
MetaTableAccessor.parseRegionInfoFromRegionName(regionNameOrEncodedRegionName);
if (regionInfo.isMetaRegion()) {
future = connection.registry.getMetaRegionLocation()
.thenApply(locs -> Stream.of(locs.getRegionLocations())
.filter(loc -> loc.getRegion().getReplicaId() == regionInfo.getReplicaId())
.findFirst());
} else {
future =
AsyncMetaTableAccessor.getRegionLocation(metaTable, regionNameOrEncodedRegionName);
}
}
CompletableFuture<HRegionLocation> returnedFuture = new CompletableFuture<>();

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletableFuture;
@ -83,7 +84,7 @@ public abstract class TestAsyncAdminBase {
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 120000);
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
TEST_UTIL.getConfiguration().setInt(START_LOG_ERRORS_AFTER_COUNT_KEY, 0);
TEST_UTIL.startMiniCluster(2);
TEST_UTIL.startMiniCluster(3);
ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
}
@ -103,36 +104,54 @@ public abstract class TestAsyncAdminBase {
@After
public void tearDown() throws Exception {
admin.listTableNames(Pattern.compile(tableName.getNameAsString() + ".*"), false)
.whenCompleteAsync((tables, err) -> {
if (tables != null) {
tables.forEach(table -> {
try {
admin.disableTable(table).join();
} catch (Exception e) {
LOG.debug("Table: " + tableName + " already disabled, so just deleting it.");
}
admin.deleteTable(table).join();
});
}
}, ForkJoinPool.commonPool()).join();
.whenCompleteAsync((tables, err) -> {
if (tables != null) {
tables.forEach(table -> {
try {
admin.disableTable(table).join();
} catch (Exception e) {
LOG.debug("Table: " + tableName + " already disabled, so just deleting it.");
}
admin.deleteTable(table).join();
});
}
}, ForkJoinPool.commonPool()).join();
}
protected void createTableWithDefaultConf(TableName tableName) {
protected void createTableWithDefaultConf(TableName tableName) throws IOException {
createTableWithDefaultConf(tableName, null);
}
protected void createTableWithDefaultConf(TableName tableName, byte[][] splitKeys) {
protected void createTableWithDefaultConf(TableName tableName, int regionReplication)
throws IOException {
createTableWithDefaultConf(tableName, regionReplication, null, FAMILY);
}
protected void createTableWithDefaultConf(TableName tableName, byte[][] splitKeys)
throws IOException {
createTableWithDefaultConf(tableName, splitKeys, FAMILY);
}
protected void createTableWithDefaultConf(TableName tableName, int regionReplication,
byte[][] splitKeys) throws IOException {
createTableWithDefaultConf(tableName, regionReplication, splitKeys, FAMILY);
}
protected void createTableWithDefaultConf(TableName tableName, byte[][] splitKeys,
byte[]... families) {
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
byte[]... families) throws IOException {
createTableWithDefaultConf(tableName, 1, splitKeys, families);
}
protected void createTableWithDefaultConf(TableName tableName, int regionReplication,
byte[][] splitKeys, byte[]... families) throws IOException {
TableDescriptorBuilder builder =
TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(regionReplication);
for (byte[] family : families) {
builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
}
CompletableFuture<Void> future = splitKeys == null ? admin.createTable(builder.build())
: admin.createTable(builder.build(), splitKeys);
: admin.createTable(builder.build(), splitKeys);
future.join();
TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
}
}

View File

@ -196,7 +196,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
}
regionServerCount.incrementAndGet();
});
assertEquals(2, regionServerCount.get());
assertEquals(3, regionServerCount.get());
}
@Test

View File

@ -184,7 +184,7 @@ public class TestAsyncRegionAdminApi2 extends TestAsyncAdminBase {
}
@Test
public void testMergeRegionsInvalidRegionCount() throws InterruptedException {
public void testMergeRegionsInvalidRegionCount() throws Exception {
byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") };
createTableWithDefaultConf(tableName, splitRows);
List<RegionInfo> regions = admin.getRegions(tableName).join();

View File

@ -0,0 +1,128 @@
/**
* 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.client;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertThat;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
@Category({ LargeTests.class, ClientTests.class })
public class TestAsyncRegionAdminApiWithRegionReplicas extends TestAsyncAdminBase {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestAsyncRegionAdminApiWithRegionReplicas.class);
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
TestAsyncAdminBase.setUpBeforeClass();
try (AsyncRegistry registry = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration())) {
RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(registry, 3);
}
}
private void testMoveNonDefaultReplica(TableName tableName)
throws InterruptedException, ExecutionException {
AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(tableName);
List<HRegionLocation> locs = locator.getAllRegionLocations().get();
// try region name
admin.move(locs.get(1).getRegion().getRegionName()).get();
assertNotEquals(locs.get(1).getServerName(),
locator.getRegionLocation(HConstants.EMPTY_START_ROW, 1, true).get());
// try encoded region name
admin.move(locs.get(2).getRegion().getEncodedNameAsBytes()).get();
assertNotEquals(locs.get(2).getServerName(),
locator.getRegionLocation(HConstants.EMPTY_START_ROW, 2, true).get());
}
@Test
public void testMoveNonDefaultReplica()
throws InterruptedException, ExecutionException, IOException {
createTableWithDefaultConf(tableName, 3);
testMoveNonDefaultReplica(tableName);
testMoveNonDefaultReplica(TableName.META_TABLE_NAME);
}
@Test
public void testSplitNonDefaultReplica()
throws InterruptedException, ExecutionException, IOException {
createTableWithDefaultConf(tableName, 3);
List<HRegionLocation> locs =
ASYNC_CONN.getRegionLocator(tableName).getAllRegionLocations().get();
try {
admin.splitRegion(locs.get(1).getRegion().getRegionName()).get();
} catch (ExecutionException e) {
assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
}
try {
admin.splitRegion(locs.get(2).getRegion().getEncodedNameAsBytes()).get();
} catch (ExecutionException e) {
assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
}
}
@Test
public void testMergeNonDefaultReplicas()
throws InterruptedException, ExecutionException, IOException {
byte[][] splitRows = new byte[][] { Bytes.toBytes(0) };
createTableWithDefaultConf(tableName, 3, splitRows);
List<HRegionLocation> locs =
ASYNC_CONN.getRegionLocator(tableName).getAllRegionLocations().get();
assertEquals(6, locs.size());
Map<Integer, List<RegionInfo>> replicaId2RegionInfo = locs.stream()
.map(HRegionLocation::getRegion).collect(Collectors.groupingBy(RegionInfo::getReplicaId));
List<RegionInfo> replicaOnes = replicaId2RegionInfo.get(1);
try {
admin
.mergeRegions(replicaOnes.get(0).getRegionName(), replicaOnes.get(1).getRegionName(), false)
.get();
} catch (ExecutionException e) {
assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
}
List<RegionInfo> replicaTwos = replicaId2RegionInfo.get(2);
try {
admin
.mergeRegions(replicaTwos.get(0).getRegionName(), replicaTwos.get(1).getRegionName(), false)
.get();
} catch (ExecutionException e) {
assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
}
}
}