HBASE-22121 AsyncAdmin can not deal with non default meta replica
Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
parent
833736f28a
commit
7fe31b060f
|
@ -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<>();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -196,7 +196,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
}
|
||||
regionServerCount.incrementAndGet();
|
||||
});
|
||||
assertEquals(2, regionServerCount.get());
|
||||
assertEquals(3, regionServerCount.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue