HBASE-22040 Add mergeRegionsAsync with a List of region names method in AsyncAdmin
Signed-off-by: Zheng Hu <openinx@gmail.com>
This commit is contained in:
parent
6cd78e899f
commit
cc55835d03
|
@ -830,29 +830,31 @@ public interface Admin extends Abortable, Closeable {
|
|||
|
||||
/**
|
||||
* Merge two regions. Asynchronous operation.
|
||||
*
|
||||
* @param nameOfRegionA encoded or full name of region a
|
||||
* @param nameOfRegionB encoded or full name of region b
|
||||
* @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge
|
||||
* two adjacent regions
|
||||
* @throws IOException
|
||||
* @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge two
|
||||
* adjacent regions
|
||||
*/
|
||||
Future<Void> mergeRegionsAsync(
|
||||
byte[] nameOfRegionA,
|
||||
byte[] nameOfRegionB,
|
||||
boolean forcible) throws IOException;
|
||||
default Future<Void> mergeRegionsAsync(byte[] nameOfRegionA, byte[] nameOfRegionB,
|
||||
boolean forcible) throws IOException {
|
||||
byte[][] nameofRegionsToMerge = new byte[2][];
|
||||
nameofRegionsToMerge[0] = nameOfRegionA;
|
||||
nameofRegionsToMerge[1] = nameOfRegionB;
|
||||
return mergeRegionsAsync(nameofRegionsToMerge, forcible);
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge regions. Asynchronous operation.
|
||||
*
|
||||
* <p/>
|
||||
* You may get a {@code DoNotRetryIOException} if you pass more than two regions in but the master
|
||||
* does not support merging more than two regions. At least till 2.2.0, we still only support
|
||||
* merging two regions.
|
||||
* @param nameofRegionsToMerge encoded or full name of daughter regions
|
||||
* @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge
|
||||
* adjacent regions
|
||||
* @throws IOException
|
||||
*/
|
||||
Future<Void> mergeRegionsAsync(
|
||||
byte[][] nameofRegionsToMerge,
|
||||
boolean forcible) throws IOException;
|
||||
Future<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Split a table. The method will execute split action for each region in table.
|
||||
|
|
|
@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
|
|||
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
|
@ -498,8 +499,22 @@ public interface AsyncAdmin {
|
|||
* @param forcible true if do a compulsory merge, otherwise we will only merge two adjacent
|
||||
* regions
|
||||
*/
|
||||
CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
|
||||
boolean forcible);
|
||||
default CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
|
||||
boolean forcible) {
|
||||
return mergeRegions(Arrays.asList(nameOfRegionA, nameOfRegionB), forcible);
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge regions.
|
||||
* <p/>
|
||||
* You may get a {@code DoNotRetryIOException} if you pass more than two regions in but the master
|
||||
* does not support merging more than two regions. At least till 2.2.0, we still only support
|
||||
* merging two regions.
|
||||
* @param nameOfRegionsToMerge encoded or full name of daughter regions
|
||||
* @param forcible true if do a compulsory merge, otherwise we will only merge two adjacent
|
||||
* regions
|
||||
*/
|
||||
CompletableFuture<Void> mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible);
|
||||
|
||||
/**
|
||||
* Split a table. The method will execute split action for each region in table.
|
||||
|
|
|
@ -316,9 +316,8 @@ class AsyncHBaseAdmin implements AsyncAdmin {
|
|||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
|
||||
boolean forcible) {
|
||||
return wrap(rawAdmin.mergeRegions(nameOfRegionA, nameOfRegionB, forcible));
|
||||
public CompletableFuture<Void> mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible) {
|
||||
return wrap(rawAdmin.mergeRegions(nameOfRegionsToMerge, forcible));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -106,6 +106,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
@ -1423,42 +1424,22 @@ public class HBaseAdmin implements Admin {
|
|||
TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge two regions. Asynchronous operation.
|
||||
* @param nameOfRegionA encoded or full name of region a
|
||||
* @param nameOfRegionB encoded or full name of region b
|
||||
* @param forcible true if do a compulsory merge, otherwise we will only merge
|
||||
* two adjacent regions
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public Future<Void> mergeRegionsAsync(
|
||||
final byte[] nameOfRegionA,
|
||||
final byte[] nameOfRegionB,
|
||||
final boolean forcible) throws IOException {
|
||||
byte[][] nameofRegionsToMerge = new byte[2][];
|
||||
nameofRegionsToMerge[0] = nameOfRegionA;
|
||||
nameofRegionsToMerge[1] = nameOfRegionB;
|
||||
return mergeRegionsAsync(nameofRegionsToMerge, forcible);
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge two regions. Asynchronous operation.
|
||||
* @param nameofRegionsToMerge encoded or full name of daughter regions
|
||||
* @param forcible true if do a compulsory merge, otherwise we will only merge
|
||||
* adjacent regions
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public Future<Void> mergeRegionsAsync(
|
||||
final byte[][] nameofRegionsToMerge,
|
||||
final boolean forcible) throws IOException {
|
||||
assert(nameofRegionsToMerge.length >= 2);
|
||||
public Future<Void> mergeRegionsAsync(final byte[][] nameofRegionsToMerge, final boolean forcible)
|
||||
throws IOException {
|
||||
Preconditions.checkArgument(nameofRegionsToMerge.length >= 2, "Can not merge only %s region",
|
||||
nameofRegionsToMerge.length);
|
||||
byte[][] encodedNameofRegionsToMerge = new byte[nameofRegionsToMerge.length][];
|
||||
for(int i = 0; i < nameofRegionsToMerge.length; i++) {
|
||||
encodedNameofRegionsToMerge[i] = RegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ?
|
||||
nameofRegionsToMerge[i] :
|
||||
Bytes.toBytes(RegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
|
||||
for (int i = 0; i < nameofRegionsToMerge.length; i++) {
|
||||
encodedNameofRegionsToMerge[i] =
|
||||
RegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ? nameofRegionsToMerge[i]
|
||||
: Bytes.toBytes(RegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
|
||||
}
|
||||
|
||||
TableName tableName = null;
|
||||
|
|
|
@ -1165,13 +1165,12 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
});
|
||||
}
|
||||
|
||||
private CompletableFuture<TableName> checkRegionsAndGetTableName(byte[] encodeRegionNameA,
|
||||
byte[] encodeRegionNameB) {
|
||||
private CompletableFuture<TableName> checkRegionsAndGetTableName(byte[][] encodedRegionNames) {
|
||||
AtomicReference<TableName> tableNameRef = new AtomicReference<>();
|
||||
CompletableFuture<TableName> future = new CompletableFuture<>();
|
||||
|
||||
checkAndGetTableName(encodeRegionNameA, tableNameRef, future);
|
||||
checkAndGetTableName(encodeRegionNameB, tableNameRef, future);
|
||||
for (byte[] encodedRegionName : encodedRegionNames) {
|
||||
checkAndGetTableName(encodedRegionName, tableNameRef, future);
|
||||
}
|
||||
return future;
|
||||
}
|
||||
|
||||
|
@ -1221,41 +1220,42 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
|
||||
boolean forcible) {
|
||||
public CompletableFuture<Void> mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible) {
|
||||
if (nameOfRegionsToMerge.size() < 2) {
|
||||
return failedFuture(new IllegalArgumentException(
|
||||
"Can not merge only " + nameOfRegionsToMerge.size() + " region"));
|
||||
}
|
||||
CompletableFuture<Void> future = new CompletableFuture<>();
|
||||
final byte[] encodeRegionNameA = toEncodeRegionName(nameOfRegionA);
|
||||
final byte[] encodeRegionNameB = toEncodeRegionName(nameOfRegionB);
|
||||
byte[][] encodedNameOfRegionsToMerge =
|
||||
nameOfRegionsToMerge.stream().map(this::toEncodeRegionName).toArray(byte[][]::new);
|
||||
|
||||
addListener(checkRegionsAndGetTableName(encodeRegionNameA, encodeRegionNameB),
|
||||
(tableName, err) -> {
|
||||
if (err != null) {
|
||||
future.completeExceptionally(err);
|
||||
return;
|
||||
}
|
||||
addListener(checkRegionsAndGetTableName(encodedNameOfRegionsToMerge), (tableName, err) -> {
|
||||
if (err != null) {
|
||||
future.completeExceptionally(err);
|
||||
return;
|
||||
}
|
||||
|
||||
MergeTableRegionsRequest request = null;
|
||||
try {
|
||||
request = RequestConverter.buildMergeTableRegionsRequest(
|
||||
new byte[][] { encodeRegionNameA, encodeRegionNameB }, forcible, ng.getNonceGroup(),
|
||||
ng.newNonce());
|
||||
} catch (DeserializationException e) {
|
||||
future.completeExceptionally(e);
|
||||
return;
|
||||
}
|
||||
MergeTableRegionsRequest request = null;
|
||||
try {
|
||||
request = RequestConverter.buildMergeTableRegionsRequest(encodedNameOfRegionsToMerge,
|
||||
forcible, ng.getNonceGroup(), ng.newNonce());
|
||||
} catch (DeserializationException e) {
|
||||
future.completeExceptionally(e);
|
||||
return;
|
||||
}
|
||||
|
||||
addListener(
|
||||
this.<MergeTableRegionsRequest, MergeTableRegionsResponse> procedureCall(tableName,
|
||||
request, (s, c, req, done) -> s.mergeTableRegions(c, req, done),
|
||||
(resp) -> resp.getProcId(), new MergeTableRegionProcedureBiConsumer(tableName)),
|
||||
(ret, err2) -> {
|
||||
if (err2 != null) {
|
||||
future.completeExceptionally(err2);
|
||||
} else {
|
||||
future.complete(ret);
|
||||
}
|
||||
});
|
||||
});
|
||||
addListener(
|
||||
this.<MergeTableRegionsRequest, MergeTableRegionsResponse> procedureCall(tableName, request,
|
||||
(s, c, req, done) -> s.mergeTableRegions(c, req, done), (resp) -> resp.getProcId(),
|
||||
new MergeTableRegionProcedureBiConsumer(tableName)),
|
||||
(ret, err2) -> {
|
||||
if (err2 != null) {
|
||||
future.completeExceptionally(err2);
|
||||
} else {
|
||||
future.complete(ret);
|
||||
}
|
||||
});
|
||||
});
|
||||
return future;
|
||||
}
|
||||
|
||||
|
|
|
@ -794,7 +794,10 @@ public class MasterRpcServices extends RSRpcServices
|
|||
|
||||
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
|
||||
|
||||
assert(request.getRegionCount() == 2);
|
||||
if (request.getRegionCount() != 2) {
|
||||
throw new ServiceException(new DoNotRetryIOException(
|
||||
"Only support merging 2 regions but " + request.getRegionCount() + " region passed"));
|
||||
}
|
||||
RegionInfo[] regionsToMerge = new RegionInfo[request.getRegionCount()];
|
||||
for (int i = 0; i < request.getRegionCount(); i++) {
|
||||
final byte[] encodedNameOfRegion = request.getRegion(i).getValue().toByteArray();
|
||||
|
|
|
@ -29,9 +29,11 @@ import java.util.HashMap;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.regex.Pattern;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
|
@ -1418,6 +1420,50 @@ public class TestAdmin1 {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergeRegionsInvalidRegionCount()
|
||||
throws IOException, InterruptedException, ExecutionException {
|
||||
TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
|
||||
.setColumnFamily(ColumnFamilyDescriptorBuilder.of("d")).build();
|
||||
byte[][] splitRows = new byte[2][];
|
||||
splitRows[0] = new byte[] { (byte) '3' };
|
||||
splitRows[1] = new byte[] { (byte) '6' };
|
||||
try {
|
||||
TEST_UTIL.createTable(td, splitRows);
|
||||
TEST_UTIL.waitTableAvailable(tableName);
|
||||
|
||||
List<RegionInfo> tableRegions = ADMIN.getRegions(tableName);
|
||||
// 0
|
||||
try {
|
||||
ADMIN.mergeRegionsAsync(new byte[0][0], false).get();
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
// expected
|
||||
}
|
||||
// 1
|
||||
try {
|
||||
ADMIN.mergeRegionsAsync(new byte[][] { tableRegions.get(0).getEncodedNameAsBytes() }, false)
|
||||
.get();
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
// expected
|
||||
}
|
||||
// 3
|
||||
try {
|
||||
ADMIN.mergeRegionsAsync(
|
||||
tableRegions.stream().map(RegionInfo::getEncodedNameAsBytes).toArray(byte[][]::new),
|
||||
false).get();
|
||||
fail();
|
||||
} catch (DoNotRetryIOException e) {
|
||||
// expected
|
||||
}
|
||||
} finally {
|
||||
ADMIN.disableTable(tableName);
|
||||
ADMIN.deleteTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSplitShouldNotHappenIfSplitIsDisabledForTable()
|
||||
throws Exception {
|
||||
|
|
|
@ -17,7 +17,22 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertThat;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
|
@ -33,15 +48,6 @@ import org.junit.experimental.categories.Category;
|
|||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* Class to test asynchronous region admin operations.
|
||||
* @see TestAsyncRegionAdminApi This test and it used to be joined it was taking longer than our
|
||||
|
@ -177,6 +183,40 @@ public class TestAsyncRegionAdminApi2 extends TestAsyncAdminBase {
|
|||
assertEquals(1, regionLocations.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergeRegionsInvalidRegionCount() throws InterruptedException {
|
||||
byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") };
|
||||
createTableWithDefaultConf(tableName, splitRows);
|
||||
List<RegionInfo> regions = admin.getRegions(tableName).join();
|
||||
// 0
|
||||
try {
|
||||
admin.mergeRegions(Collections.emptyList(), false).get();
|
||||
fail();
|
||||
} catch (ExecutionException e) {
|
||||
// expected
|
||||
assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
|
||||
}
|
||||
// 1
|
||||
try {
|
||||
admin.mergeRegions(regions.stream().limit(1).map(RegionInfo::getEncodedNameAsBytes)
|
||||
.collect(Collectors.toList()), false).get();
|
||||
fail();
|
||||
} catch (ExecutionException e) {
|
||||
// expected
|
||||
assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
|
||||
}
|
||||
// 3
|
||||
try {
|
||||
admin.mergeRegions(
|
||||
regions.stream().map(RegionInfo::getEncodedNameAsBytes).collect(Collectors.toList()), false)
|
||||
.get();
|
||||
fail();
|
||||
} catch (ExecutionException e) {
|
||||
// expected
|
||||
assertThat(e.getCause(), instanceOf(DoNotRetryIOException.class));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSplitTable() throws Exception {
|
||||
initSplitMergeSwitch();
|
||||
|
|
Loading…
Reference in New Issue