HBASE-18283 Provide a construct method which accept a thread pool for AsyncAdmin
This commit is contained in:
parent
8318a092ac
commit
14f0423b58
|
@ -46,11 +46,6 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
@InterfaceAudience.Public
|
||||
public interface AsyncAdmin {
|
||||
|
||||
/**
|
||||
* @return Async Connection used by this object.
|
||||
*/
|
||||
AsyncConnectionImpl getConnection();
|
||||
|
||||
/**
|
||||
* @param tableName Table to check.
|
||||
* @return True if table exists already. The return value will be wrapped by a
|
||||
|
@ -105,7 +100,9 @@ public interface AsyncAdmin {
|
|||
* Creates a new table.
|
||||
* @param desc table descriptor for table
|
||||
*/
|
||||
CompletableFuture<Void> createTable(TableDescriptor desc);
|
||||
default CompletableFuture<Void> createTable(TableDescriptor desc) {
|
||||
return createTable(desc, Optional.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new table with the specified number of regions. The start key specified will become
|
||||
|
@ -128,7 +125,7 @@ public interface AsyncAdmin {
|
|||
* @param desc table descriptor for table
|
||||
* @param splitKeys array of split keys for the initial regions of the table
|
||||
*/
|
||||
CompletableFuture<Void> createTable(TableDescriptor desc, byte[][] splitKeys);
|
||||
CompletableFuture<Void> createTable(TableDescriptor desc, Optional<byte[][]> splitKeys);
|
||||
|
||||
/**
|
||||
* Deletes a table.
|
||||
|
@ -186,6 +183,13 @@ public interface AsyncAdmin {
|
|||
*/
|
||||
CompletableFuture<List<TableDescriptor>> disableTables(Pattern pattern);
|
||||
|
||||
/**
|
||||
* @param tableName name of table to check
|
||||
* @return true if table is on-line. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
*/
|
||||
CompletableFuture<Boolean> isTableEnabled(TableName tableName);
|
||||
|
||||
/**
|
||||
* @param tableName name of table to check
|
||||
* @return true if table is off-line. The return value will be wrapped by a
|
||||
|
@ -198,7 +202,9 @@ public interface AsyncAdmin {
|
|||
* @return true if all regions of the table are available. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
*/
|
||||
CompletableFuture<Boolean> isTableAvailable(TableName tableName);
|
||||
default CompletableFuture<Boolean> isTableAvailable(TableName tableName) {
|
||||
return isTableAvailable(tableName, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Use this api to check if the table has been created with the specified number of splitkeys
|
||||
|
@ -274,13 +280,6 @@ public interface AsyncAdmin {
|
|||
*/
|
||||
CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors();
|
||||
|
||||
/**
|
||||
* @param tableName name of table to check
|
||||
* @return true if table is on-line. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
*/
|
||||
CompletableFuture<Boolean> isTableEnabled(TableName tableName);
|
||||
|
||||
/**
|
||||
* Turn the load balancer on or off.
|
||||
* @param on
|
||||
|
@ -330,7 +329,7 @@ public interface AsyncAdmin {
|
|||
/**
|
||||
* Get all the online regions on a region server.
|
||||
*/
|
||||
CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName sn);
|
||||
CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName);
|
||||
|
||||
/**
|
||||
* Flush a table.
|
||||
|
@ -422,15 +421,15 @@ public interface AsyncAdmin {
|
|||
|
||||
/**
|
||||
* Compact all regions on the region server.
|
||||
* @param sn the region server name
|
||||
* @param serverName the region server name
|
||||
*/
|
||||
CompletableFuture<Void> compactRegionServer(ServerName sn);
|
||||
CompletableFuture<Void> compactRegionServer(ServerName serverName);
|
||||
|
||||
/**
|
||||
* Compact all regions on the region server.
|
||||
* @param sn the region server name
|
||||
* @param serverName the region server name
|
||||
*/
|
||||
CompletableFuture<Void> majorCompactRegionServer(ServerName sn);
|
||||
CompletableFuture<Void> majorCompactRegionServer(ServerName serverName);
|
||||
|
||||
/**
|
||||
* Merge two regions.
|
||||
|
@ -563,18 +562,18 @@ public interface AsyncAdmin {
|
|||
|
||||
/**
|
||||
* Append the replicable table-cf config of the specified peer
|
||||
* @param id a short that identifies the cluster
|
||||
* @param peerId a short that identifies the cluster
|
||||
* @param tableCfs A map from tableName to column family names
|
||||
*/
|
||||
CompletableFuture<Void> appendReplicationPeerTableCFs(String id,
|
||||
CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId,
|
||||
Map<TableName, ? extends Collection<String>> tableCfs);
|
||||
|
||||
/**
|
||||
* Remove some table-cfs from config of the specified peer
|
||||
* @param id a short name that identifies the cluster
|
||||
* @param peerId a short name that identifies the cluster
|
||||
* @param tableCfs A map from tableName to column family names
|
||||
*/
|
||||
CompletableFuture<Void> removeReplicationPeerTableCFs(String id,
|
||||
CompletableFuture<Void> removeReplicationPeerTableCFs(String peerId,
|
||||
Map<TableName, ? extends Collection<String>> tableCfs);
|
||||
|
||||
/**
|
||||
|
@ -613,7 +612,9 @@ public interface AsyncAdmin {
|
|||
* @param snapshotName name of the snapshot to be created
|
||||
* @param tableName name of the table for which snapshot is created
|
||||
*/
|
||||
CompletableFuture<Void> snapshot(String snapshotName, TableName tableName);
|
||||
default CompletableFuture<Void> snapshot(String snapshotName, TableName tableName) {
|
||||
return snapshot(snapshotName, tableName, SnapshotType.FLUSH);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
|
||||
|
@ -627,8 +628,10 @@ public interface AsyncAdmin {
|
|||
* @param tableName name of the table to snapshot
|
||||
* @param type type of snapshot to take
|
||||
*/
|
||||
CompletableFuture<Void> snapshot(String snapshotName, TableName tableName,
|
||||
SnapshotType type);
|
||||
default CompletableFuture<Void> snapshot(String snapshotName, TableName tableName,
|
||||
SnapshotType type) {
|
||||
return snapshot(new SnapshotDescription(snapshotName, tableName, type));
|
||||
}
|
||||
|
||||
/**
|
||||
* Take a snapshot and wait for the server to complete that snapshot asynchronously. Only a single
|
||||
|
@ -695,14 +698,16 @@ public interface AsyncAdmin {
|
|||
* @return a list of snapshot descriptors for completed snapshots wrapped by a
|
||||
* {@link CompletableFuture}
|
||||
*/
|
||||
CompletableFuture<List<SnapshotDescription>> listSnapshots();
|
||||
default CompletableFuture<List<SnapshotDescription>> listSnapshots() {
|
||||
return listSnapshots(Optional.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* List all the completed snapshots matching the given pattern.
|
||||
* @param pattern The compiled regular expression to match against
|
||||
* @return - returns a List of SnapshotDescription wrapped by a {@link CompletableFuture}
|
||||
*/
|
||||
CompletableFuture<List<SnapshotDescription>> listSnapshots(Pattern pattern);
|
||||
CompletableFuture<List<SnapshotDescription>> listSnapshots(Optional<Pattern> pattern);
|
||||
|
||||
/**
|
||||
* List all the completed snapshots matching the given table name regular expression and snapshot
|
||||
|
@ -725,7 +730,9 @@ public interface AsyncAdmin {
|
|||
* Delete existing snapshots whose names match the pattern passed.
|
||||
* @param pattern pattern for names of the snapshot to match
|
||||
*/
|
||||
CompletableFuture<Void> deleteSnapshots(Pattern pattern);
|
||||
default CompletableFuture<Void> deleteSnapshots(Pattern pattern) {
|
||||
return deleteTableSnapshots(null, pattern);
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete all existing snapshots matching the given table name regular expression and snapshot
|
||||
|
|
|
@ -96,11 +96,17 @@ public interface AsyncConnection extends Closeable {
|
|||
AsyncTableBuilder<AsyncTable> getTableBuilder(TableName tableName, ExecutorService pool);
|
||||
|
||||
/**
|
||||
* Retrieve an AsyncAdmin implementation to administer an HBase cluster. The returned AsyncAdmin
|
||||
* is not guaranteed to be thread-safe. A new instance should be created for each using thread.
|
||||
* This is a lightweight operation. Pooling or caching of the returned AsyncAdmin is not
|
||||
* recommended.
|
||||
* Retrieve an AsyncAdmin implementation to administer an HBase cluster. The returned
|
||||
* {@code CompletableFuture} will be finished directly in the rpc framework's callback thread, so
|
||||
* typically you should not do any time consuming work inside these methods.
|
||||
* @return an AsyncAdmin instance for cluster administration
|
||||
*/
|
||||
AsyncAdmin getAdmin();
|
||||
|
||||
/**
|
||||
* Retrieve an AsyncAdmin implementation to administer an HBase cluster.
|
||||
* @param pool the thread pool to use for executing callback
|
||||
* @return an AsyncAdmin instance for cluster administration
|
||||
*/
|
||||
AsyncAdmin getAdmin(ExecutorService pool);
|
||||
}
|
||||
|
|
|
@ -279,6 +279,11 @@ class AsyncConnectionImpl implements AsyncConnection {
|
|||
|
||||
@Override
|
||||
public AsyncAdmin getAdmin() {
|
||||
return new AsyncHBaseAdmin(this);
|
||||
return new RawAsyncHBaseAdmin(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncAdmin getAdmin(ExecutorService pool) {
|
||||
return new AsyncHBaseAdmin(new RawAsyncHBaseAdmin(this), pool);
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -19,11 +19,13 @@ package org.apache.hadoop.hbase.shaded.protobuf;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -1271,19 +1273,26 @@ public final class RequestConverter {
|
|||
final byte [][] splitKeys,
|
||||
final long nonceGroup,
|
||||
final long nonce) {
|
||||
return buildCreateTableRequest(hTableDesc, Optional.ofNullable(splitKeys), nonceGroup, nonce);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a protocol buffer CreateTableRequest
|
||||
* @param hTableDesc
|
||||
* @param splitKeys
|
||||
* @return a CreateTableRequest
|
||||
*/
|
||||
public static CreateTableRequest buildCreateTableRequest(TableDescriptor hTableDesc,
|
||||
Optional<byte[][]> splitKeys, long nonceGroup, long nonce) {
|
||||
CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
|
||||
builder.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDesc));
|
||||
if (splitKeys != null) {
|
||||
for (byte [] splitKey : splitKeys) {
|
||||
builder.addSplitKeys(UnsafeByteOperations.unsafeWrap(splitKey));
|
||||
}
|
||||
}
|
||||
splitKeys.ifPresent(keys -> Arrays.stream(keys).forEach(
|
||||
key -> builder.addSplitKeys(UnsafeByteOperations.unsafeWrap(key))));
|
||||
builder.setNonceGroup(nonceGroup);
|
||||
builder.setNonce(nonce);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a protocol buffer ModifyTableRequest
|
||||
*
|
||||
|
|
|
@ -19,15 +19,32 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ForkJoinPool;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.rules.TestName;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameter;
|
||||
import org.junit.runners.Parameterized.Parameters;
|
||||
|
||||
/**
|
||||
* Class to test AsyncAdmin.
|
||||
|
@ -36,13 +53,34 @@ public abstract class TestAsyncAdminBase {
|
|||
|
||||
protected static final Log LOG = LogFactory.getLog(TestAsyncAdminBase.class);
|
||||
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
protected static byte[] FAMILY = Bytes.toBytes("testFamily");
|
||||
protected static final byte[] FAMILY = Bytes.toBytes("testFamily");
|
||||
protected static final byte[] FAMILY_0 = Bytes.toBytes("cf0");
|
||||
protected static final byte[] FAMILY_1 = Bytes.toBytes("cf1");
|
||||
|
||||
protected static AsyncConnection ASYNC_CONN;
|
||||
protected AsyncAdmin admin;
|
||||
|
||||
@Parameter
|
||||
public Supplier<AsyncAdmin> getAdmin;
|
||||
|
||||
private static AsyncAdmin getRawAsyncAdmin() {
|
||||
return ASYNC_CONN.getAdmin();
|
||||
}
|
||||
|
||||
private static AsyncAdmin getAsyncAdmin() {
|
||||
return ASYNC_CONN.getAdmin(ForkJoinPool.commonPool());
|
||||
}
|
||||
|
||||
@Parameters
|
||||
public static List<Object[]> params() {
|
||||
return Arrays.asList(new Supplier<?>[] { TestAsyncAdminBase::getRawAsyncAdmin },
|
||||
new Supplier<?>[] { TestAsyncAdminBase::getAsyncAdmin });
|
||||
}
|
||||
|
||||
@Rule
|
||||
public TestName testName = new TestName();
|
||||
protected TableName tableName;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 60000);
|
||||
|
@ -60,7 +98,43 @@ public abstract class TestAsyncAdminBase {
|
|||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
this.admin = ASYNC_CONN.getAdmin();
|
||||
public void setUp() {
|
||||
admin = ASYNC_CONN.getAdmin();
|
||||
String methodName = testName.getMethodName();
|
||||
tableName = TableName.valueOf(methodName.substring(0, methodName.length() - 3));
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() {
|
||||
admin.listTableNames(Optional.of(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();
|
||||
}
|
||||
|
||||
protected void createTableWithDefaultConf(TableName tableName) {
|
||||
createTableWithDefaultConf(tableName, Optional.empty());
|
||||
}
|
||||
|
||||
protected void createTableWithDefaultConf(TableName tableName, Optional<byte[][]> splitKeys) {
|
||||
createTableWithDefaultConf(tableName, splitKeys, FAMILY);
|
||||
}
|
||||
|
||||
protected void createTableWithDefaultConf(TableName tableName, Optional<byte[][]> splitKeys,
|
||||
byte[]... families) {
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
for (byte[] family : families) {
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).build());
|
||||
}
|
||||
admin.createTable(builder.build(), splitKeys).join();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,10 @@ import org.apache.hadoop.hbase.testclassification.ClientTests;
|
|||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
@Category({ MediumTests.class, ClientTests.class })
|
||||
public class TestAsyncBalancerAdminApi extends TestAsyncAdminBase {
|
||||
|
||||
|
|
|
@ -42,10 +42,13 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
|
|||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
/**
|
||||
* Class to test asynchronous namespace admin operations.
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
@Category({ LargeTests.class, ClientTests.class })
|
||||
public class TestAsyncNamespaceAdminApi extends TestAsyncAdminBase {
|
||||
|
||||
|
|
|
@ -66,7 +66,7 @@ public class TestAsyncProcedureAdminApi extends TestAsyncAdminBase {
|
|||
|
||||
@Test
|
||||
public void testExecProcedure() throws Exception {
|
||||
TableName tableName = TableName.valueOf("testExecProcedure");
|
||||
String snapshotString = "offlineTableSnapshot";
|
||||
try {
|
||||
Table table = TEST_UTIL.createTable(tableName, Bytes.toBytes("cf"));
|
||||
for (int i = 0; i < 100; i++) {
|
||||
|
@ -74,13 +74,13 @@ public class TestAsyncProcedureAdminApi extends TestAsyncAdminBase {
|
|||
table.put(put);
|
||||
}
|
||||
// take a snapshot of the enabled table
|
||||
String snapshotString = "offlineTableSnapshot";
|
||||
Map<String, String> props = new HashMap<>();
|
||||
props.put("table", tableName.getNameAsString());
|
||||
admin.execProcedure(SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION, snapshotString,
|
||||
props).get();
|
||||
LOG.debug("Snapshot completed.");
|
||||
} finally {
|
||||
admin.deleteSnapshot(snapshotString).join();
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,10 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaCache;
|
||||
|
@ -34,50 +30,35 @@ import org.apache.hadoop.hbase.quotas.ThrottleType;
|
|||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
@Category({ ClientTests.class, MediumTests.class })
|
||||
public class TestAsyncQuotaAdminApi {
|
||||
private static final Log LOG = LogFactory.getLog(TestAsyncQuotaAdminApi.class);
|
||||
|
||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
protected static AsyncConnection ASYNC_CONN;
|
||||
protected AsyncAdmin admin;
|
||||
public class TestAsyncQuotaAdminApi extends TestAsyncAdminBase {
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
|
||||
TEST_UTIL.getConfiguration().setInt(QuotaCache.REFRESH_CONF_KEY, 2000);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
|
||||
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 60000);
|
||||
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(1);
|
||||
TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME);
|
||||
ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
IOUtils.closeQuietly(ASYNC_CONN);
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
this.admin = ASYNC_CONN.getAdmin();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testThrottleType() throws Exception {
|
||||
String userName = User.getCurrent().getShortName();
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
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;
|
||||
|
@ -31,11 +32,10 @@ import java.util.concurrent.ExecutionException;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
|
@ -51,36 +51,29 @@ import org.apache.hadoop.hbase.testclassification.ClientTests;
|
|||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
/**
|
||||
* Class to test asynchronous region admin operations.
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
@Category({ LargeTests.class, ClientTests.class })
|
||||
public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
||||
|
||||
public static Random RANDOM = new Random(System.currentTimeMillis());
|
||||
|
||||
private void createTableWithDefaultConf(TableName TABLENAME) throws Exception {
|
||||
HTableDescriptor htd = new HTableDescriptor(TABLENAME);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("value");
|
||||
htd.addFamily(hcd);
|
||||
|
||||
admin.createTable(htd, null).get();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloseRegion() throws Exception {
|
||||
TableName TABLENAME = TableName.valueOf("TestHBACloseRegion");
|
||||
createTableWithDefaultConf(TABLENAME);
|
||||
createTableWithDefaultConf(tableName);
|
||||
|
||||
HRegionInfo info = null;
|
||||
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
|
||||
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
|
||||
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
|
||||
for (HRegionInfo regionInfo : onlineRegions) {
|
||||
if (!regionInfo.getTable().isSystemTable()) {
|
||||
|
@ -102,16 +95,14 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
|
||||
@Test
|
||||
public void testCloseRegionIfInvalidRegionNameIsPassed() throws Exception {
|
||||
final String name = "TestHBACloseRegion1";
|
||||
byte[] TABLENAME = Bytes.toBytes(name);
|
||||
createTableWithDefaultConf(TableName.valueOf(TABLENAME));
|
||||
createTableWithDefaultConf(tableName);
|
||||
|
||||
HRegionInfo info = null;
|
||||
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(TABLENAME));
|
||||
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
|
||||
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
|
||||
for (HRegionInfo regionInfo : onlineRegions) {
|
||||
if (!regionInfo.isMetaTable()) {
|
||||
if (regionInfo.getRegionNameAsString().contains(name)) {
|
||||
if (regionInfo.getRegionNameAsString().contains(tableName.getNameAsString())) {
|
||||
info = regionInfo;
|
||||
boolean catchNotServingException = false;
|
||||
try {
|
||||
|
@ -132,10 +123,9 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
|
||||
@Test
|
||||
public void testCloseRegionWhenServerNameIsEmpty() throws Exception {
|
||||
byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegionWhenServerNameIsEmpty");
|
||||
createTableWithDefaultConf(TableName.valueOf(TABLENAME));
|
||||
createTableWithDefaultConf(tableName);
|
||||
|
||||
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(TABLENAME));
|
||||
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
|
||||
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
|
||||
for (HRegionInfo regionInfo : onlineRegions) {
|
||||
if (!regionInfo.isMetaTable()) {
|
||||
|
@ -147,125 +137,22 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetRegion() throws Exception {
|
||||
AsyncHBaseAdmin rawAdmin = (AsyncHBaseAdmin) admin;
|
||||
|
||||
final TableName tableName = TableName.valueOf("testGetRegion");
|
||||
LOG.info("Started " + tableName);
|
||||
public void testGetRegionLocation() throws Exception {
|
||||
RawAsyncHBaseAdmin rawAdmin = (RawAsyncHBaseAdmin) ASYNC_CONN.getAdmin();
|
||||
TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
|
||||
|
||||
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
|
||||
HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm"));
|
||||
AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(tableName);
|
||||
HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm")).get();
|
||||
HRegionInfo region = regionLocation.getRegionInfo();
|
||||
byte[] regionName = region.getRegionName();
|
||||
byte[] regionName = regionLocation.getRegionInfo().getRegionName();
|
||||
HRegionLocation location = rawAdmin.getRegionLocation(regionName).get();
|
||||
assertTrue(Bytes.equals(regionName, location.getRegionInfo().getRegionName()));
|
||||
location = rawAdmin.getRegionLocation(region.getEncodedNameAsBytes()).get();
|
||||
assertTrue(Bytes.equals(regionName, location.getRegionInfo().getRegionName()));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergeRegions() throws Exception {
|
||||
final TableName tableName = TableName.valueOf("testMergeRegions");
|
||||
HColumnDescriptor cd = new HColumnDescriptor("d");
|
||||
HTableDescriptor td = new HTableDescriptor(tableName);
|
||||
td.addFamily(cd);
|
||||
byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") };
|
||||
Admin syncAdmin = TEST_UTIL.getAdmin();
|
||||
try {
|
||||
TEST_UTIL.createTable(td, splitRows);
|
||||
TEST_UTIL.waitTableAvailable(tableName);
|
||||
|
||||
List<HRegionInfo> tableRegions;
|
||||
HRegionInfo regionA;
|
||||
HRegionInfo regionB;
|
||||
|
||||
// merge with full name
|
||||
tableRegions = syncAdmin.getTableRegions(tableName);
|
||||
assertEquals(3, syncAdmin.getTableRegions(tableName).size());
|
||||
regionA = tableRegions.get(0);
|
||||
regionB = tableRegions.get(1);
|
||||
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
|
||||
|
||||
assertEquals(2, syncAdmin.getTableRegions(tableName).size());
|
||||
|
||||
// merge with encoded name
|
||||
tableRegions = syncAdmin.getTableRegions(tableName);
|
||||
regionA = tableRegions.get(0);
|
||||
regionB = tableRegions.get(1);
|
||||
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
|
||||
|
||||
assertEquals(1, syncAdmin.getTableRegions(tableName).size());
|
||||
} finally {
|
||||
syncAdmin.disableTable(tableName);
|
||||
syncAdmin.deleteTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSplitTable() throws Exception {
|
||||
splitTests(TableName.valueOf("testSplitTable"), 3000, false, null);
|
||||
splitTests(TableName.valueOf("testSplitTableWithSplitPoint"), 3000, false, Bytes.toBytes("3"));
|
||||
splitTests(TableName.valueOf("testSplitRegion"), 3000, true, null);
|
||||
splitTests(TableName.valueOf("testSplitRegionWithSplitPoint"), 3000, true, Bytes.toBytes("3"));
|
||||
}
|
||||
|
||||
private void splitTests(TableName tableName, int rowCount, boolean isSplitRegion,
|
||||
byte[] splitPoint) throws Exception {
|
||||
int count = 0;
|
||||
// create table
|
||||
HColumnDescriptor cd = new HColumnDescriptor("d");
|
||||
HTableDescriptor td = new HTableDescriptor(tableName);
|
||||
td.addFamily(cd);
|
||||
Table table = TEST_UTIL.createTable(td, null);
|
||||
TEST_UTIL.waitTableAvailable(tableName);
|
||||
|
||||
List<HRegionInfo> regions = TEST_UTIL.getAdmin().getTableRegions(tableName);
|
||||
assertEquals(regions.size(), 1);
|
||||
|
||||
List<Put> puts = new ArrayList<>();
|
||||
for (int i = 0; i < rowCount; i++) {
|
||||
Put put = new Put(Bytes.toBytes(i));
|
||||
put.addColumn(Bytes.toBytes("d"), null, Bytes.toBytes("value" + i));
|
||||
puts.add(put);
|
||||
}
|
||||
table.put(puts);
|
||||
|
||||
if (isSplitRegion) {
|
||||
admin.splitRegion(regions.get(0).getRegionName(), Optional.ofNullable(splitPoint)).get();
|
||||
} else {
|
||||
if (splitPoint == null) {
|
||||
admin.split(tableName).get();
|
||||
} else {
|
||||
admin.split(tableName, splitPoint).get();
|
||||
}
|
||||
}
|
||||
|
||||
for (int i = 0; i < 45; i++) {
|
||||
try {
|
||||
List<HRegionInfo> hRegionInfos = TEST_UTIL.getAdmin().getTableRegions(tableName);
|
||||
count = hRegionInfos.size();
|
||||
if (count >= 2) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(1000L);
|
||||
} catch (Exception e) {
|
||||
LOG.error(e);
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(count, 2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAssignRegionAndUnassignRegion() throws Exception {
|
||||
final TableName tableName = TableName.valueOf("testAssignRegionAndUnassignRegion");
|
||||
try {
|
||||
// create test table
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.addFamily(new HColumnDescriptor(FAMILY));
|
||||
admin.createTable(desc).get();
|
||||
createTableWithDefaultConf(tableName);
|
||||
|
||||
// assign region.
|
||||
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
|
||||
|
@ -298,15 +185,13 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
// Expected
|
||||
}
|
||||
assertTrue(regionStates.getRegionState(hri).isClosed());
|
||||
} finally {
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
HRegionInfo createTableAndGetOneRegion(final TableName tableName)
|
||||
throws IOException, InterruptedException, ExecutionException {
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.addFamily(new HColumnDescriptor(FAMILY));
|
||||
TableDescriptor desc =
|
||||
TableDescriptorBuilder.newBuilder(tableName)
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()).build();
|
||||
admin.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), 5).get();
|
||||
|
||||
// wait till the table is assigned
|
||||
|
@ -333,8 +218,6 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
// Will cause the Master to tell the regionserver to shut itself down because
|
||||
// regionserver is reporting the state as OPEN.
|
||||
public void testOfflineRegion() throws Exception {
|
||||
final TableName tableName = TableName.valueOf("testOfflineRegion");
|
||||
try {
|
||||
HRegionInfo hri = createTableAndGetOneRegion(tableName);
|
||||
|
||||
RegionStates regionStates =
|
||||
|
@ -344,8 +227,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
long timeoutTime = System.currentTimeMillis() + 3000;
|
||||
while (true) {
|
||||
if (regionStates.getRegionByStateOfTable(tableName).get(RegionState.State.OFFLINE)
|
||||
.contains(hri))
|
||||
break;
|
||||
.contains(hri)) break;
|
||||
long now = System.currentTimeMillis();
|
||||
if (now > timeoutTime) {
|
||||
fail("Failed to offline the region in time");
|
||||
|
@ -355,39 +237,29 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
}
|
||||
RegionState regionState = regionStates.getRegionState(hri);
|
||||
assertTrue(regionState.isOffline());
|
||||
} finally {
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRegionByStateOfTable() throws Exception {
|
||||
final TableName tableName = TableName.valueOf("testGetRegionByStateOfTable");
|
||||
try {
|
||||
HRegionInfo hri = createTableAndGetOneRegion(tableName);
|
||||
|
||||
RegionStates regionStates =
|
||||
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
|
||||
assertTrue(regionStates.getRegionByStateOfTable(tableName)
|
||||
.get(RegionState.State.OPEN)
|
||||
assertTrue(regionStates.getRegionByStateOfTable(tableName).get(RegionState.State.OPEN)
|
||||
.contains(hri));
|
||||
assertFalse(regionStates.getRegionByStateOfTable(TableName.valueOf("I_am_the_phantom"))
|
||||
.get(RegionState.State.OPEN)
|
||||
.contains(hri));
|
||||
} finally {
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
}
|
||||
.get(RegionState.State.OPEN).contains(hri));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMoveRegion() throws Exception {
|
||||
final TableName tableName = TableName.valueOf("testMoveRegion");
|
||||
try {
|
||||
admin.setBalancerOn(false).join();
|
||||
|
||||
HRegionInfo hri = createTableAndGetOneRegion(tableName);
|
||||
RawAsyncHBaseAdmin rawAdmin = (RawAsyncHBaseAdmin) ASYNC_CONN.getAdmin();
|
||||
ServerName serverName = rawAdmin.getRegionLocation(hri.getRegionName()).get().getServerName();
|
||||
|
||||
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
|
||||
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
|
||||
ServerName serverName = regionStates.getRegionServerOfRegion(hri);
|
||||
ServerManager serverManager = master.getServerManager();
|
||||
ServerName destServerName = null;
|
||||
List<JVMClusterUtil.RegionServerThread> regionServers =
|
||||
|
@ -399,66 +271,63 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
break;
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(destServerName != null && !destServerName.equals(serverName));
|
||||
admin.move(hri.getRegionName(), Optional.of(destServerName)).get();
|
||||
|
||||
long timeoutTime = System.currentTimeMillis() + 30000;
|
||||
while (true) {
|
||||
ServerName sn = regionStates.getRegionServerOfRegion(hri);
|
||||
ServerName sn = rawAdmin.getRegionLocation(hri.getRegionName()).get().getServerName();
|
||||
if (sn != null && sn.equals(destServerName)) {
|
||||
TEST_UTIL.assertRegionOnServer(hri, sn, 200);
|
||||
break;
|
||||
}
|
||||
long now = System.currentTimeMillis();
|
||||
if (now > timeoutTime) {
|
||||
fail("Failed to move the region in time: " + regionStates.getRegionState(hri));
|
||||
fail("Failed to move the region in time: " + hri);
|
||||
}
|
||||
regionStates.wait(50);
|
||||
}
|
||||
} finally {
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
Thread.sleep(100);
|
||||
}
|
||||
admin.setBalancerOn(true).join();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetOnlineRegions() throws Exception {
|
||||
final TableName tableName = TableName.valueOf("testGetOnlineRegions");
|
||||
try {
|
||||
createTableAndGetOneRegion(tableName);
|
||||
AtomicInteger regionServerCount = new AtomicInteger(0);
|
||||
TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().stream()
|
||||
.map(rsThread -> rsThread.getRegionServer().getServerName()).forEach(serverName -> {
|
||||
TEST_UTIL
|
||||
.getHBaseCluster()
|
||||
.getLiveRegionServerThreads()
|
||||
.stream()
|
||||
.map(rsThread -> rsThread.getRegionServer())
|
||||
.forEach(
|
||||
rs -> {
|
||||
ServerName serverName = rs.getServerName();
|
||||
try {
|
||||
Assert.assertEquals(admin.getOnlineRegions(serverName).get().size(),
|
||||
TEST_UTIL.getAdmin().getOnlineRegions(serverName).size());
|
||||
Assert.assertEquals(admin.getOnlineRegions(serverName).get().size(), rs
|
||||
.getOnlineRegions().size());
|
||||
} catch (Exception e) {
|
||||
fail("admin.getOnlineRegions() method throws a exception: " + e.getMessage());
|
||||
}
|
||||
regionServerCount.incrementAndGet();
|
||||
});
|
||||
Assert.assertEquals(regionServerCount.get(), 2);
|
||||
} catch (Exception e) {
|
||||
LOG.info("Exception", e);
|
||||
throw e;
|
||||
} finally {
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFlushTableAndRegion() throws Exception {
|
||||
final TableName tableName = TableName.valueOf("testFlushRegion");
|
||||
try {
|
||||
HRegionInfo hri = createTableAndGetOneRegion(tableName);
|
||||
ServerName serverName = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
|
||||
.getRegionStates().getRegionServerOfRegion(hri);
|
||||
HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().stream()
|
||||
ServerName serverName =
|
||||
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
|
||||
.getRegionServerOfRegion(hri);
|
||||
HRegionServer regionServer =
|
||||
TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().stream()
|
||||
.map(rsThread -> rsThread.getRegionServer())
|
||||
.filter(rs -> rs.getServerName().equals(serverName)).findFirst().get();
|
||||
|
||||
// write a put into the specific region
|
||||
try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
|
||||
table.put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-1")));
|
||||
}
|
||||
ASYNC_CONN.getRawTable(tableName)
|
||||
.put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-1")))
|
||||
.join();
|
||||
Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemstoreSize() > 0);
|
||||
// flush region and wait flush operation finished.
|
||||
LOG.info("flushing region: " + Bytes.toStringBinary(hri.getRegionName()));
|
||||
|
@ -472,9 +341,9 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
Assert.assertEquals(regionServer.getOnlineRegion(hri.getRegionName()).getMemstoreSize(), 0);
|
||||
|
||||
// write another put into the specific region
|
||||
try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
|
||||
table.put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-2")));
|
||||
}
|
||||
ASYNC_CONN.getRawTable(tableName)
|
||||
.put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-2")))
|
||||
.join();
|
||||
Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemstoreSize() > 0);
|
||||
admin.flush(tableName).get();
|
||||
Threads.sleepWithoutInterrupt(500);
|
||||
|
@ -483,92 +352,173 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
}
|
||||
// check the memstore.
|
||||
Assert.assertEquals(regionServer.getOnlineRegion(hri.getRegionName()).getMemstoreSize(), 0);
|
||||
} finally {
|
||||
TEST_UTIL.deleteTable(tableName);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergeRegions() throws Exception {
|
||||
byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") };
|
||||
createTableWithDefaultConf(tableName, Optional.of(splitRows));
|
||||
|
||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
||||
List<HRegionLocation> regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
HRegionInfo regionA;
|
||||
HRegionInfo regionB;
|
||||
|
||||
// merge with full name
|
||||
assertEquals(3, regionLocations.size());
|
||||
regionA = regionLocations.get(0).getRegionInfo();
|
||||
regionB = regionLocations.get(1).getRegionInfo();
|
||||
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
|
||||
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
assertEquals(2, regionLocations.size());
|
||||
// merge with encoded name
|
||||
regionA = regionLocations.get(0).getRegionInfo();
|
||||
regionB = regionLocations.get(1).getRegionInfo();
|
||||
admin.mergeRegions(regionA.getRegionName(), regionB.getRegionName(), false).get();
|
||||
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
assertEquals(1, regionLocations.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSplitTable() throws Exception {
|
||||
splitTest(TableName.valueOf("testSplitTable"), 3000, false, null);
|
||||
splitTest(TableName.valueOf("testSplitTableWithSplitPoint"), 3000, false, Bytes.toBytes("3"));
|
||||
splitTest(TableName.valueOf("testSplitTableRegion"), 3000, true, null);
|
||||
splitTest(TableName.valueOf("testSplitTableRegionWithSplitPoint2"), 3000, true, Bytes.toBytes("3"));
|
||||
}
|
||||
|
||||
private void
|
||||
splitTest(TableName tableName, int rowCount, boolean isSplitRegion, byte[] splitPoint)
|
||||
throws Exception {
|
||||
// create table
|
||||
createTableWithDefaultConf(tableName);
|
||||
|
||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
||||
List<HRegionLocation> regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
assertEquals(1, regionLocations.size());
|
||||
|
||||
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
|
||||
List<Put> puts = new ArrayList<>();
|
||||
for (int i = 0; i < rowCount; i++) {
|
||||
Put put = new Put(Bytes.toBytes(i));
|
||||
put.addColumn(FAMILY, null, Bytes.toBytes("value" + i));
|
||||
puts.add(put);
|
||||
}
|
||||
table.putAll(puts).join();
|
||||
|
||||
if (isSplitRegion) {
|
||||
admin.splitRegion(regionLocations.get(0).getRegionInfo().getRegionName(),
|
||||
Optional.ofNullable(splitPoint)).get();
|
||||
} else {
|
||||
if (splitPoint == null) {
|
||||
admin.split(tableName).get();
|
||||
} else {
|
||||
admin.split(tableName, splitPoint).get();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 600000)
|
||||
public void testCompactRpcAPI() throws Exception {
|
||||
String tableName = "testCompactRpcAPI";
|
||||
compactionTest(tableName, 8, CompactionState.MAJOR, false);
|
||||
compactionTest(tableName, 15, CompactionState.MINOR, false);
|
||||
compactionTest(tableName, 8, CompactionState.MAJOR, true);
|
||||
compactionTest(tableName, 15, CompactionState.MINOR, true);
|
||||
}
|
||||
|
||||
@Test(timeout = 600000)
|
||||
public void testCompactRegionServer() throws Exception {
|
||||
TableName table = TableName.valueOf("testCompactRegionServer");
|
||||
byte[][] families = { Bytes.toBytes("f1"), Bytes.toBytes("f2"), Bytes.toBytes("f3") };
|
||||
Table ht = null;
|
||||
int count = 0;
|
||||
for (int i = 0; i < 45; i++) {
|
||||
try {
|
||||
ht = TEST_UTIL.createTable(table, families);
|
||||
loadData(ht, families, 3000, 8);
|
||||
List<HRegionServer> rsList = TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().stream()
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName))
|
||||
.get();
|
||||
count = regionLocations.size();
|
||||
if (count >= 2) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(1000L);
|
||||
} catch (Exception e) {
|
||||
LOG.error(e);
|
||||
}
|
||||
}
|
||||
assertEquals(count, 2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompactRegionServer() throws Exception {
|
||||
byte[][] families = { Bytes.toBytes("f1"), Bytes.toBytes("f2"), Bytes.toBytes("f3") };
|
||||
createTableWithDefaultConf(tableName, Optional.empty(), families);
|
||||
loadData(tableName, families, 3000, 8);
|
||||
|
||||
List<HRegionServer> rsList =
|
||||
TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().stream()
|
||||
.map(rsThread -> rsThread.getRegionServer()).collect(Collectors.toList());
|
||||
List<Region> regions = new ArrayList<>();
|
||||
rsList.forEach(rs -> regions.addAll(rs.getOnlineRegions(table)));
|
||||
rsList.forEach(rs -> regions.addAll(rs.getOnlineRegions(tableName)));
|
||||
Assert.assertEquals(regions.size(), 1);
|
||||
int countBefore = countStoreFilesInFamilies(regions, families);
|
||||
Assert.assertTrue(countBefore > 0);
|
||||
|
||||
// Minor compaction for all region servers.
|
||||
for (HRegionServer rs : rsList)
|
||||
admin.compactRegionServer(rs.getServerName()).get();
|
||||
Thread.sleep(5000);
|
||||
int countAfterMinorCompaction = countStoreFilesInFamilies(regions, families);
|
||||
Assert.assertTrue(countAfterMinorCompaction < countBefore);
|
||||
|
||||
// Major compaction for all region servers.
|
||||
for (HRegionServer rs : rsList)
|
||||
admin.majorCompactRegionServer(rs.getServerName()).get();
|
||||
Thread.sleep(5000);
|
||||
int countAfterMajorCompaction = countStoreFilesInFamilies(regions, families);
|
||||
Assert.assertEquals(countAfterMajorCompaction, 3);
|
||||
} finally {
|
||||
if (ht != null) {
|
||||
TEST_UTIL.deleteTable(table);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void compactionTest(final String tableName, final int flushes,
|
||||
@Test
|
||||
public void testCompact() throws Exception {
|
||||
compactionTest(TableName.valueOf("testCompact1"), 8, CompactionState.MAJOR, false);
|
||||
compactionTest(TableName.valueOf("testCompact2"), 15, CompactionState.MINOR, false);
|
||||
compactionTest(TableName.valueOf("testCompact3"), 8, CompactionState.MAJOR, true);
|
||||
compactionTest(TableName.valueOf("testCompact4"), 15, CompactionState.MINOR, true);
|
||||
}
|
||||
|
||||
private void compactionTest(final TableName tableName, final int flushes,
|
||||
final CompactionState expectedState, boolean singleFamily) throws Exception {
|
||||
// Create a table with regions
|
||||
final TableName table = TableName.valueOf(tableName);
|
||||
byte[] family = Bytes.toBytes("family");
|
||||
byte[][] families =
|
||||
{ family, Bytes.add(family, Bytes.toBytes("2")), Bytes.add(family, Bytes.toBytes("3")) };
|
||||
Table ht = null;
|
||||
try {
|
||||
ht = TEST_UTIL.createTable(table, families);
|
||||
loadData(ht, families, 3000, flushes);
|
||||
createTableWithDefaultConf(tableName, Optional.empty(), families);
|
||||
loadData(tableName, families, 3000, flushes);
|
||||
|
||||
List<Region> regions = new ArrayList<>();
|
||||
TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads()
|
||||
.forEach(rsThread -> regions.addAll(rsThread.getRegionServer().getOnlineRegions(table)));
|
||||
TEST_UTIL
|
||||
.getHBaseCluster()
|
||||
.getLiveRegionServerThreads()
|
||||
.forEach(rsThread -> regions.addAll(rsThread.getRegionServer().getOnlineRegions(tableName)));
|
||||
Assert.assertEquals(regions.size(), 1);
|
||||
|
||||
int countBefore = countStoreFilesInFamilies(regions, families);
|
||||
int countBeforeSingleFamily = countStoreFilesInFamily(regions, family);
|
||||
assertTrue(countBefore > 0); // there should be some data files
|
||||
if (expectedState == CompactionState.MINOR) {
|
||||
if (singleFamily) {
|
||||
admin.compact(table, Optional.of(family)).get();
|
||||
admin.compact(tableName, Optional.of(family)).get();
|
||||
} else {
|
||||
admin.compact(table, Optional.empty()).get();
|
||||
admin.compact(tableName, Optional.empty()).get();
|
||||
}
|
||||
} else {
|
||||
if (singleFamily) {
|
||||
admin.majorCompact(table, Optional.of(family)).get();
|
||||
admin.majorCompact(tableName, Optional.of(family)).get();
|
||||
} else {
|
||||
admin.majorCompact(table, Optional.empty()).get();
|
||||
admin.majorCompact(tableName, Optional.empty()).get();
|
||||
}
|
||||
}
|
||||
|
||||
long curt = System.currentTimeMillis();
|
||||
long waitTime = 5000;
|
||||
long endt = curt + waitTime;
|
||||
CompactionState state = TEST_UTIL.getAdmin().getCompactionState(table);
|
||||
CompactionState state = TEST_UTIL.getAdmin().getCompactionState(tableName);
|
||||
while (state == CompactionState.NONE && curt < endt) {
|
||||
Thread.sleep(10);
|
||||
state = TEST_UTIL.getAdmin().getCompactionState(table);
|
||||
state = TEST_UTIL.getAdmin().getCompactionState(tableName);
|
||||
curt = System.currentTimeMillis();
|
||||
}
|
||||
// Now, should have the right compaction state,
|
||||
|
@ -580,14 +530,15 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
}
|
||||
} else {
|
||||
// Wait until the compaction is done
|
||||
state = TEST_UTIL.getAdmin().getCompactionState(table);
|
||||
state = TEST_UTIL.getAdmin().getCompactionState(tableName);
|
||||
while (state != CompactionState.NONE && curt < endt) {
|
||||
Thread.sleep(10);
|
||||
state = TEST_UTIL.getAdmin().getCompactionState(table);
|
||||
state = TEST_UTIL.getAdmin().getCompactionState(tableName);
|
||||
}
|
||||
// Now, compaction should be done.
|
||||
assertEquals(CompactionState.NONE, state);
|
||||
}
|
||||
|
||||
int countAfter = countStoreFilesInFamilies(regions, families);
|
||||
int countAfterSingleFamily = countStoreFilesInFamily(regions, family);
|
||||
assertTrue(countAfter < countBefore);
|
||||
|
@ -604,11 +555,6 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
assertTrue(1 < countAfterSingleFamily);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (ht != null) {
|
||||
TEST_UTIL.deleteTable(table);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static int countStoreFilesInFamily(List<Region> regions, final byte[] family) {
|
||||
|
@ -623,8 +569,9 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
return count;
|
||||
}
|
||||
|
||||
private static void loadData(final Table ht, final byte[][] families, final int rows,
|
||||
private static void loadData(final TableName tableName, final byte[][] families, final int rows,
|
||||
final int flushes) throws IOException {
|
||||
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
|
||||
List<Put> puts = new ArrayList<>(rows);
|
||||
byte[] qualifier = Bytes.toBytes("val");
|
||||
for (int i = 0; i < flushes; i++) {
|
||||
|
@ -636,7 +583,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
}
|
||||
puts.add(p);
|
||||
}
|
||||
ht.put(puts);
|
||||
table.putAll(puts).join();
|
||||
TEST_UTIL.flush();
|
||||
puts.clear();
|
||||
}
|
||||
|
|
|
@ -45,10 +45,13 @@ import org.junit.Rule;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
/**
|
||||
* Class to test asynchronous replication admin operations.
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
@Category({LargeTests.class, ClientTests.class})
|
||||
public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
|
||||
|
||||
|
@ -57,9 +60,6 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
|
|||
private final String ID_SECOND = "2";
|
||||
private final String KEY_SECOND = "127.0.0.1:2181:/hbase2";
|
||||
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 60000);
|
||||
|
@ -142,12 +142,12 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
|
|||
public void testAppendPeerTableCFs() throws Exception {
|
||||
ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
|
||||
rpc1.setClusterKey(KEY_ONE);
|
||||
final TableName tableName1 = TableName.valueOf(name.getMethodName() + "t1");
|
||||
final TableName tableName2 = TableName.valueOf(name.getMethodName() + "t2");
|
||||
final TableName tableName3 = TableName.valueOf(name.getMethodName() + "t3");
|
||||
final TableName tableName4 = TableName.valueOf(name.getMethodName() + "t4");
|
||||
final TableName tableName5 = TableName.valueOf(name.getMethodName() + "t5");
|
||||
final TableName tableName6 = TableName.valueOf(name.getMethodName() + "t6");
|
||||
final TableName tableName1 = TableName.valueOf(tableName.getNameAsString() + "t1");
|
||||
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "t2");
|
||||
final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "t3");
|
||||
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "t4");
|
||||
final TableName tableName5 = TableName.valueOf(tableName.getNameAsString() + "t5");
|
||||
final TableName tableName6 = TableName.valueOf(tableName.getNameAsString() + "t6");
|
||||
|
||||
// Add a valid peer
|
||||
admin.addReplicationPeer(ID_ONE, rpc1).join();
|
||||
|
@ -244,10 +244,10 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
|
|||
public void testRemovePeerTableCFs() throws Exception {
|
||||
ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
|
||||
rpc1.setClusterKey(KEY_ONE);
|
||||
final TableName tableName1 = TableName.valueOf(name.getMethodName() + "t1");
|
||||
final TableName tableName2 = TableName.valueOf(name.getMethodName() + "t2");
|
||||
final TableName tableName3 = TableName.valueOf(name.getMethodName() + "t3");
|
||||
final TableName tableName4 = TableName.valueOf(name.getMethodName() + "t4");
|
||||
final TableName tableName1 = TableName.valueOf(tableName.getNameAsString() + "t1");
|
||||
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "t2");
|
||||
final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "t3");
|
||||
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "t4");
|
||||
// Add a valid peer
|
||||
admin.addReplicationPeer(ID_ONE, rpc1).join();
|
||||
Map<TableName, List<String>> tableCFs = new HashMap<>();
|
||||
|
@ -360,8 +360,8 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
|
|||
public void testNamespacesAndTableCfsConfigConflict() throws Exception {
|
||||
String ns1 = "ns1";
|
||||
String ns2 = "ns2";
|
||||
final TableName tableName1 = TableName.valueOf(ns1 + ":" + name.getMethodName());
|
||||
final TableName tableName2 = TableName.valueOf(ns2 + ":" + name.getMethodName() + "2");
|
||||
final TableName tableName1 = TableName.valueOf(ns1 + ":" + tableName.getNameAsString() + "1");
|
||||
final TableName tableName2 = TableName.valueOf(ns2 + ":" + tableName.getNameAsString() + "2");
|
||||
|
||||
ReplicationPeerConfig rpc = new ReplicationPeerConfig();
|
||||
rpc.setClusterKey(KEY_ONE);
|
||||
|
|
|
@ -29,12 +29,16 @@ import org.junit.Rule;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
@Category({ LargeTests.class, ClientTests.class })
|
||||
public class TestAsyncSnapshotAdminApi extends TestAsyncAdminBase {
|
||||
|
||||
|
@ -42,15 +46,6 @@ public class TestAsyncSnapshotAdminApi extends TestAsyncAdminBase {
|
|||
String snapshotName2 = "snapshotName2";
|
||||
String snapshotName3 = "snapshotName3";
|
||||
|
||||
@Rule
|
||||
public TestName testName = new TestName();
|
||||
TableName tableName;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
tableName = TableName.valueOf(testName.getMethodName());
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanup() throws Exception {
|
||||
admin.deleteSnapshots(Pattern.compile(".*")).get();
|
||||
|
@ -175,10 +170,13 @@ public class TestAsyncSnapshotAdminApi extends TestAsyncAdminBase {
|
|||
admin.snapshot(snapshotName3, tableName).get();
|
||||
Assert.assertEquals(admin.listSnapshots().get().size(), 3);
|
||||
|
||||
Assert.assertEquals(admin.listSnapshots(Pattern.compile("(.*)")).get().size(), 3);
|
||||
Assert.assertEquals(admin.listSnapshots(Pattern.compile("snapshotName(\\d+)")).get().size(), 3);
|
||||
Assert.assertEquals(admin.listSnapshots(Pattern.compile("snapshotName[1|3]")).get().size(), 2);
|
||||
Assert.assertEquals(admin.listSnapshots(Pattern.compile("snapshot(.*)")).get().size(), 3);
|
||||
Assert.assertEquals(admin.listSnapshots(Optional.of(Pattern.compile("(.*)"))).get().size(), 3);
|
||||
Assert.assertEquals(admin.listSnapshots(Optional.of(Pattern.compile("snapshotName(\\d+)")))
|
||||
.get().size(), 3);
|
||||
Assert.assertEquals(admin.listSnapshots(Optional.of(Pattern.compile("snapshotName[1|3]")))
|
||||
.get().size(), 2);
|
||||
Assert.assertEquals(admin.listSnapshots(Optional.of(Pattern.compile("snapshot(.*)"))).get()
|
||||
.size(), 3);
|
||||
Assert.assertEquals(
|
||||
admin.listTableSnapshots(Pattern.compile("testListSnapshots"), Pattern.compile("s(.*)")).get()
|
||||
.size(),
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
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;
|
||||
|
@ -32,11 +33,11 @@ import java.util.Map;
|
|||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletionException;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
|
@ -44,6 +45,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotEnabledException;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
|
@ -51,23 +53,20 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
/**
|
||||
* Class to test asynchronous table admin operations.
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
@Category({LargeTests.class, ClientTests.class})
|
||||
public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
||||
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
||||
@Test
|
||||
public void testTableExist() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
boolean exist;
|
||||
exist = admin.tableExists(tableName).get();
|
||||
assertEquals(false, exist);
|
||||
|
@ -81,12 +80,12 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
@Test
|
||||
public void testListTables() throws Exception {
|
||||
int numTables = admin.listTables().get().size();
|
||||
final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1");
|
||||
final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2");
|
||||
final TableName tableName3 = TableName.valueOf(name.getMethodName() + "3");
|
||||
final TableName tableName1 = TableName.valueOf(tableName.getNameAsString() + "1");
|
||||
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "2");
|
||||
final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "3");
|
||||
TableName[] tables = new TableName[] { tableName1, tableName2, tableName3 };
|
||||
for (int i = 0; i < tables.length; i++) {
|
||||
TEST_UTIL.createTable(tables[i], FAMILY);
|
||||
createTableWithDefaultConf(tables[i]);
|
||||
}
|
||||
|
||||
List<TableDescriptor> tableDescs = admin.listTables().get();
|
||||
|
@ -118,7 +117,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
}
|
||||
|
||||
for (int i = 0; i < tables.length; i++) {
|
||||
TEST_UTIL.deleteTable(tables[i]);
|
||||
admin.disableTable(tables[i]).join();
|
||||
admin.deleteTable(tables[i]).join();
|
||||
}
|
||||
|
||||
tableDescs = admin.listTables(Optional.empty(), true).get();
|
||||
|
@ -127,27 +127,25 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
assertTrue("Not found system tables", tableNames.size() > 0);
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
@Test
|
||||
public void testGetTableDescriptor() throws Exception {
|
||||
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
|
||||
HColumnDescriptor fam2 = new HColumnDescriptor("fam2");
|
||||
HColumnDescriptor fam3 = new HColumnDescriptor("fam3");
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
htd.addFamily(fam1);
|
||||
htd.addFamily(fam2);
|
||||
htd.addFamily(fam3);
|
||||
admin.createTable(htd).join();
|
||||
TableDescriptor confirmedHtd = admin.getTableDescriptor(htd.getTableName()).get();
|
||||
assertEquals(htd.compareTo(new HTableDescriptor(confirmedHtd)), 0);
|
||||
byte[][] families = { FAMILY, FAMILY_0, FAMILY_1 };
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
for (byte[] family : families) {
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).build());
|
||||
}
|
||||
TableDescriptor desc = builder.build();
|
||||
admin.createTable(desc).join();
|
||||
ModifyableTableDescriptor modifyableDesc = ((ModifyableTableDescriptor) desc);
|
||||
TableDescriptor confirmedHtd = admin.getTableDescriptor(tableName).get();
|
||||
assertEquals(modifyableDesc.compareTo((ModifyableTableDescriptor) confirmedHtd), 0);
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
@Test
|
||||
public void testCreateTable() throws Exception {
|
||||
List<TableDescriptor> tables = admin.listTables().get();
|
||||
int numTables = tables.size();
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
admin.createTable(new HTableDescriptor(tableName).addFamily(new HColumnDescriptor(FAMILY)))
|
||||
.join();
|
||||
createTableWithDefaultConf(tableName);
|
||||
tables = admin.listTables().get();
|
||||
assertEquals(numTables + 1, tables.size());
|
||||
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster().getMaster()
|
||||
|
@ -162,84 +160,70 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
return state.get().getState();
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
@Test
|
||||
public void testCreateTableNumberOfRegions() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
admin.createTable(desc).join();
|
||||
List<HRegionLocation> regions;
|
||||
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
|
||||
regions = l.getAllRegionLocations();
|
||||
assertEquals("Table should have only 1 region", 1, regions.size());
|
||||
}
|
||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
||||
|
||||
createTableWithDefaultConf(tableName);
|
||||
List<HRegionLocation> regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
assertEquals("Table should have only 1 region", 1, regionLocations.size());
|
||||
|
||||
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "_2");
|
||||
desc = new HTableDescriptor(tableName2);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
admin.createTable(desc, new byte[][] { new byte[] { 42 } }).join();
|
||||
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName2)) {
|
||||
regions = l.getAllRegionLocations();
|
||||
assertEquals("Table should have only 2 region", 2, regions.size());
|
||||
}
|
||||
createTableWithDefaultConf(tableName2, Optional.of(new byte[][] { new byte[] { 42 } }));
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName2)).get();
|
||||
assertEquals("Table should have only 2 region", 2, regionLocations.size());
|
||||
|
||||
final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "_3");
|
||||
desc = new HTableDescriptor(tableName3);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
admin.createTable(desc, "a".getBytes(), "z".getBytes(), 3).join();
|
||||
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName3)) {
|
||||
regions = l.getAllRegionLocations();
|
||||
assertEquals("Table should have only 3 region", 3, regions.size());
|
||||
}
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName3);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build());
|
||||
admin.createTable(builder.build(), "a".getBytes(), "z".getBytes(), 3).join();
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
|
||||
assertEquals("Table should have only 3 region", 3, regionLocations.size());
|
||||
|
||||
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");
|
||||
desc = new HTableDescriptor(tableName4);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
builder = TableDescriptorBuilder.newBuilder(tableName4);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build());
|
||||
try {
|
||||
admin.createTable(desc, "a".getBytes(), "z".getBytes(), 2).join();
|
||||
admin.createTable(builder.build(), "a".getBytes(), "z".getBytes(), 2).join();
|
||||
fail("Should not be able to create a table with only 2 regions using this API.");
|
||||
} catch (CompletionException e) {
|
||||
assertTrue(e.getCause() instanceof IllegalArgumentException);
|
||||
}
|
||||
|
||||
final TableName tableName5 = TableName.valueOf(tableName.getNameAsString() + "_5");
|
||||
desc = new HTableDescriptor(tableName5);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
admin.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16).join();
|
||||
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName5)) {
|
||||
regions = l.getAllRegionLocations();
|
||||
assertEquals("Table should have 16 region", 16, regions.size());
|
||||
}
|
||||
builder = TableDescriptorBuilder.newBuilder(tableName5);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build());
|
||||
admin.createTable(builder.build(), new byte[] { 1 }, new byte[] { 127 }, 16).join();
|
||||
regionLocations =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName5)).get();
|
||||
assertEquals("Table should have 16 region", 16, regionLocations.size());
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
@Test
|
||||
public void testCreateTableWithRegions() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
|
||||
byte[][] splitKeys = { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 }, new byte[] { 3, 3, 3 },
|
||||
new byte[] { 4, 4, 4 }, new byte[] { 5, 5, 5 }, new byte[] { 6, 6, 6 },
|
||||
new byte[] { 7, 7, 7 }, new byte[] { 8, 8, 8 }, new byte[] { 9, 9, 9 }, };
|
||||
int expectedRegions = splitKeys.length + 1;
|
||||
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
admin.createTable(desc, splitKeys).join();
|
||||
createTableWithDefaultConf(tableName, Optional.of(splitKeys));
|
||||
|
||||
boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys).get();
|
||||
assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable);
|
||||
|
||||
List<HRegionLocation> regions;
|
||||
Iterator<HRegionLocation> hris;
|
||||
HRegionInfo hri;
|
||||
ClusterConnection conn = (ClusterConnection) TEST_UTIL.getConnection();
|
||||
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
|
||||
regions = l.getAllRegionLocations();
|
||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
||||
List<HRegionLocation> regions =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
Iterator<HRegionLocation> hris = regions.iterator();
|
||||
|
||||
assertEquals(
|
||||
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
||||
expectedRegions, regions.size());
|
||||
System.err.println("Found " + regions.size() + " regions");
|
||||
|
||||
HRegionInfo hri;
|
||||
hris = regions.iterator();
|
||||
hri = hris.next().getRegionInfo();
|
||||
assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
|
||||
|
@ -271,9 +255,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
hri = hris.next().getRegionInfo();
|
||||
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8]));
|
||||
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
|
||||
|
||||
verifyRoundRobinDistribution(conn, l, expectedRegions);
|
||||
}
|
||||
verifyRoundRobinDistribution(regions, expectedRegions);
|
||||
|
||||
// Now test using start/end with a number of regions
|
||||
|
||||
|
@ -283,17 +265,14 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
|
||||
// Splitting into 10 regions, we expect (null,1) ... (9, null)
|
||||
// with (1,2) (2,3) (3,4) (4,5) (5,6) (6,7) (7,8) (8,9) in the middle
|
||||
|
||||
expectedRegions = 10;
|
||||
|
||||
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "_2");
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName2);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build());
|
||||
admin.createTable(builder.build(), startKey, endKey, expectedRegions).join();
|
||||
|
||||
desc = new HTableDescriptor(tableName2);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
admin.createTable(desc, startKey, endKey, expectedRegions).join();
|
||||
|
||||
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName2)) {
|
||||
regions = l.getAllRegionLocations();
|
||||
regions =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName2)).get();
|
||||
assertEquals(
|
||||
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
||||
expectedRegions, regions.size());
|
||||
|
@ -330,52 +309,42 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
hri = hris.next().getRegionInfo();
|
||||
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }));
|
||||
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
|
||||
|
||||
verifyRoundRobinDistribution(conn, l, expectedRegions);
|
||||
}
|
||||
verifyRoundRobinDistribution(regions, expectedRegions);
|
||||
|
||||
// Try once more with something that divides into something infinite
|
||||
|
||||
startKey = new byte[] { 0, 0, 0, 0, 0, 0 };
|
||||
endKey = new byte[] { 1, 0, 0, 0, 0, 0 };
|
||||
|
||||
expectedRegions = 5;
|
||||
|
||||
final TableName tableName3 = TableName.valueOf(tableName.getNameAsString() + "_3");
|
||||
builder = TableDescriptorBuilder.newBuilder(tableName3);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build());
|
||||
admin.createTable(builder.build(), startKey, endKey, expectedRegions).join();
|
||||
|
||||
desc = new HTableDescriptor(tableName3);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
admin.createTable(desc, startKey, endKey, expectedRegions).join();
|
||||
|
||||
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName3)) {
|
||||
regions = l.getAllRegionLocations();
|
||||
regions =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName3)).get();
|
||||
assertEquals(
|
||||
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
||||
expectedRegions, regions.size());
|
||||
System.err.println("Found " + regions.size() + " regions");
|
||||
|
||||
verifyRoundRobinDistribution(conn, l, expectedRegions);
|
||||
}
|
||||
verifyRoundRobinDistribution(regions, expectedRegions);
|
||||
|
||||
// Try an invalid case where there are duplicate split keys
|
||||
splitKeys = new byte[][] { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 },
|
||||
new byte[] { 3, 3, 3 }, new byte[] { 2, 2, 2 } };
|
||||
|
||||
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");
|
||||
desc = new HTableDescriptor(tableName4);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
final TableName tableName4 = TableName.valueOf(tableName.getNameAsString() + "_4");;
|
||||
try {
|
||||
admin.createTable(desc, splitKeys).join();
|
||||
createTableWithDefaultConf(tableName4, Optional.of(splitKeys));
|
||||
fail("Should not be able to create this table because of " + "duplicate split keys");
|
||||
} catch (CompletionException e) {
|
||||
assertTrue(e.getCause() instanceof IllegalArgumentException);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyRoundRobinDistribution(ClusterConnection c, RegionLocator regionLocator,
|
||||
int expectedRegions) throws IOException {
|
||||
int numRS = c.getCurrentNrHRS();
|
||||
List<HRegionLocation> regions = regionLocator.getAllRegionLocations();
|
||||
private void verifyRoundRobinDistribution(List<HRegionLocation> regions, int expectedRegions)
|
||||
throws IOException {
|
||||
int numRS = ((ClusterConnection) TEST_UTIL.getConnection()).getCurrentNrHRS();
|
||||
|
||||
Map<ServerName, List<HRegionInfo>> server2Regions = new HashMap<>();
|
||||
regions.stream().forEach((loc) -> {
|
||||
ServerName server = loc.getServerName();
|
||||
|
@ -394,103 +363,93 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
});
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
public void testCreateTableWithOnlyEmptyStartRow() throws IOException {
|
||||
byte[] tableName = Bytes.toBytes(name.getMethodName());
|
||||
@Test
|
||||
public void testCreateTableWithOnlyEmptyStartRow() throws Exception {
|
||||
byte[][] splitKeys = new byte[1][];
|
||||
splitKeys[0] = HConstants.EMPTY_BYTE_ARRAY;
|
||||
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
|
||||
desc.addFamily(new HColumnDescriptor("col"));
|
||||
try {
|
||||
admin.createTable(desc, splitKeys).join();
|
||||
createTableWithDefaultConf(tableName, Optional.of(splitKeys));
|
||||
fail("Test case should fail as empty split key is passed.");
|
||||
} catch (CompletionException e) {
|
||||
assertTrue(e.getCause() instanceof IllegalArgumentException);
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
public void testCreateTableWithEmptyRowInTheSplitKeys() throws IOException {
|
||||
byte[] tableName = Bytes.toBytes(name.getMethodName());
|
||||
@Test
|
||||
public void testCreateTableWithEmptyRowInTheSplitKeys() throws Exception {
|
||||
byte[][] splitKeys = new byte[3][];
|
||||
splitKeys[0] = "region1".getBytes();
|
||||
splitKeys[1] = HConstants.EMPTY_BYTE_ARRAY;
|
||||
splitKeys[2] = "region2".getBytes();
|
||||
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
|
||||
desc.addFamily(new HColumnDescriptor("col"));
|
||||
try {
|
||||
admin.createTable(desc, splitKeys).join();
|
||||
createTableWithDefaultConf(tableName, Optional.of(splitKeys));
|
||||
fail("Test case should fail as empty split key is passed.");
|
||||
} catch (CompletionException e) {
|
||||
assertTrue(e.getCause() instanceof IllegalArgumentException);
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
@Test
|
||||
public void testDeleteTable() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
admin.createTable(new HTableDescriptor(tableName).addFamily(new HColumnDescriptor(FAMILY))).join();
|
||||
createTableWithDefaultConf(tableName);
|
||||
assertTrue(admin.tableExists(tableName).get());
|
||||
TEST_UTIL.getAdmin().disableTable(tableName);
|
||||
admin.deleteTable(tableName).join();
|
||||
assertFalse(admin.tableExists(tableName).get());
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
@Test
|
||||
public void testDeleteTables() throws Exception {
|
||||
TableName[] tables = { TableName.valueOf(name.getMethodName() + "1"),
|
||||
TableName.valueOf(name.getMethodName() + "2"), TableName.valueOf(name.getMethodName() + "3") };
|
||||
Arrays.stream(tables).map(HTableDescriptor::new)
|
||||
.map((table) -> table.addFamily(new HColumnDescriptor(FAMILY))).forEach((table) -> {
|
||||
admin.createTable(table).join();
|
||||
admin.tableExists(table.getTableName()).thenAccept((exist) -> assertTrue(exist)).join();
|
||||
try {
|
||||
TEST_UTIL.getAdmin().disableTable(table.getTableName());
|
||||
} catch (Exception e) {
|
||||
}
|
||||
TableName[] tables =
|
||||
{ TableName.valueOf(tableName.getNameAsString() + "1"),
|
||||
TableName.valueOf(tableName.getNameAsString() + "2"),
|
||||
TableName.valueOf(tableName.getNameAsString() + "3") };
|
||||
Arrays.stream(tables).forEach((table) -> {
|
||||
createTableWithDefaultConf(table);
|
||||
admin.tableExists(table).thenAccept((exist) -> assertTrue(exist)).join();
|
||||
admin.disableTable(table).join();
|
||||
});
|
||||
List<TableDescriptor> failed = admin.deleteTables(Pattern.compile("testDeleteTables.*")).get();
|
||||
List<TableDescriptor> failed =
|
||||
admin.deleteTables(Pattern.compile(tableName.getNameAsString() + ".*")).get();
|
||||
assertEquals(0, failed.size());
|
||||
Arrays.stream(tables).forEach((table) -> {
|
||||
admin.tableExists(table).thenAccept((exist) -> assertFalse(exist)).join();
|
||||
});
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
public void testTruncateTable() throws IOException {
|
||||
testTruncateTable(TableName.valueOf(name.getMethodName()), false);
|
||||
@Test
|
||||
public void testTruncateTable() throws Exception {
|
||||
testTruncateTable(tableName, false);
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
public void testTruncateTablePreservingSplits() throws IOException {
|
||||
testTruncateTable(TableName.valueOf(name.getMethodName()), true);
|
||||
@Test
|
||||
public void testTruncateTablePreservingSplits() throws Exception {
|
||||
testTruncateTable(tableName, true);
|
||||
}
|
||||
|
||||
private void testTruncateTable(final TableName tableName, boolean preserveSplits)
|
||||
throws IOException {
|
||||
throws Exception {
|
||||
byte[][] splitKeys = new byte[2][];
|
||||
splitKeys[0] = Bytes.toBytes(4);
|
||||
splitKeys[1] = Bytes.toBytes(8);
|
||||
|
||||
// Create & Fill the table
|
||||
Table table = TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY, splitKeys);
|
||||
try {
|
||||
TEST_UTIL.loadNumericRows(table, HConstants.CATALOG_FAMILY, 0, 10);
|
||||
assertEquals(10, TEST_UTIL.countRows(table));
|
||||
} finally {
|
||||
table.close();
|
||||
createTableWithDefaultConf(tableName, Optional.of(splitKeys));
|
||||
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
|
||||
int expectedRows = 10;
|
||||
for (int i = 0; i < expectedRows; i++) {
|
||||
byte[] data = Bytes.toBytes(String.valueOf(i));
|
||||
Put put = new Put(data);
|
||||
put.addColumn(FAMILY, null, data);
|
||||
table.put(put).join();
|
||||
}
|
||||
assertEquals(10, table.scanAll(new Scan()).get().size());
|
||||
assertEquals(3, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
|
||||
|
||||
// Truncate & Verify
|
||||
TEST_UTIL.getAdmin().disableTable(tableName);
|
||||
admin.disableTable(tableName).join();
|
||||
admin.truncateTable(tableName, preserveSplits).join();
|
||||
table = TEST_UTIL.getConnection().getTable(tableName);
|
||||
try {
|
||||
assertEquals(0, TEST_UTIL.countRows(table));
|
||||
} finally {
|
||||
table.close();
|
||||
}
|
||||
assertEquals(0, table.scanAll(new Scan()).get().size());
|
||||
if (preserveSplits) {
|
||||
assertEquals(3, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
|
||||
} else {
|
||||
|
@ -498,149 +457,147 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
@Test
|
||||
public void testDisableAndEnableTable() throws Exception {
|
||||
createTableWithDefaultConf(tableName);
|
||||
RawAsyncTable table = ASYNC_CONN.getRawTable(tableName);
|
||||
final byte[] row = Bytes.toBytes("row");
|
||||
final byte[] qualifier = Bytes.toBytes("qualifier");
|
||||
final byte[] value = Bytes.toBytes("value");
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
Table ht = TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
|
||||
Put put = new Put(row);
|
||||
put.addColumn(HConstants.CATALOG_FAMILY, qualifier, value);
|
||||
ht.put(put);
|
||||
put.addColumn(FAMILY, qualifier, value);
|
||||
table.put(put).join();
|
||||
Get get = new Get(row);
|
||||
get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
|
||||
ht.get(get);
|
||||
get.addColumn(FAMILY, qualifier);
|
||||
table.get(get).get();
|
||||
|
||||
this.admin.disableTable(ht.getName()).join();
|
||||
this.admin.disableTable(tableName).join();
|
||||
assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster().getMaster()
|
||||
.getTableStateManager().isTableState(ht.getName(), TableState.State.DISABLED));
|
||||
.getTableStateManager().isTableState(tableName, TableState.State.DISABLED));
|
||||
assertEquals(TableState.State.DISABLED, getStateFromMeta(tableName));
|
||||
|
||||
// Test that table is disabled
|
||||
get = new Get(row);
|
||||
get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
|
||||
get.addColumn(FAMILY, qualifier);
|
||||
boolean ok = false;
|
||||
try {
|
||||
ht.get(get);
|
||||
} catch (TableNotEnabledException e) {
|
||||
table.get(get).get();
|
||||
} catch (ExecutionException e) {
|
||||
ok = true;
|
||||
}
|
||||
ok = false;
|
||||
// verify that scan encounters correct exception
|
||||
Scan scan = new Scan();
|
||||
try {
|
||||
ResultScanner scanner = ht.getScanner(scan);
|
||||
Result res = null;
|
||||
do {
|
||||
res = scanner.next();
|
||||
} while (res != null);
|
||||
} catch (TableNotEnabledException e) {
|
||||
table.scanAll(new Scan()).get();
|
||||
} catch (ExecutionException e) {
|
||||
ok = true;
|
||||
}
|
||||
assertTrue(ok);
|
||||
this.admin.enableTable(tableName).join();
|
||||
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster().getMaster()
|
||||
.getTableStateManager().isTableState(ht.getName(), TableState.State.ENABLED));
|
||||
.getTableStateManager().isTableState(tableName, TableState.State.ENABLED));
|
||||
assertEquals(TableState.State.ENABLED, getStateFromMeta(tableName));
|
||||
|
||||
// Test that table is enabled
|
||||
try {
|
||||
ht.get(get);
|
||||
} catch (RetriesExhaustedException e) {
|
||||
table.get(get).get();
|
||||
} catch (Exception e) {
|
||||
ok = false;
|
||||
}
|
||||
assertTrue(ok);
|
||||
ht.close();
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
@Test
|
||||
public void testDisableAndEnableTables() throws Exception {
|
||||
final TableName tableName1 = TableName.valueOf(tableName.getNameAsString() + "1");
|
||||
final TableName tableName2 = TableName.valueOf(tableName.getNameAsString() + "2");
|
||||
createTableWithDefaultConf(tableName1);
|
||||
createTableWithDefaultConf(tableName2);
|
||||
RawAsyncTable table1 = ASYNC_CONN.getRawTable(tableName1);
|
||||
RawAsyncTable table2 = ASYNC_CONN.getRawTable(tableName1);
|
||||
|
||||
final byte[] row = Bytes.toBytes("row");
|
||||
final byte[] qualifier = Bytes.toBytes("qualifier");
|
||||
final byte[] value = Bytes.toBytes("value");
|
||||
final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1");
|
||||
final TableName tableName2 = TableName.valueOf(name.getMethodName());
|
||||
Table ht1 = TEST_UTIL.createTable(tableName1, HConstants.CATALOG_FAMILY);
|
||||
Table ht2 = TEST_UTIL.createTable(tableName2, HConstants.CATALOG_FAMILY);
|
||||
Put put = new Put(row);
|
||||
put.addColumn(HConstants.CATALOG_FAMILY, qualifier, value);
|
||||
ht1.put(put);
|
||||
ht2.put(put);
|
||||
put.addColumn(FAMILY, qualifier, value);
|
||||
table1.put(put).join();
|
||||
table2.put(put).join();
|
||||
Get get = new Get(row);
|
||||
get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
|
||||
ht1.get(get);
|
||||
ht2.get(get);
|
||||
get.addColumn(FAMILY, qualifier);
|
||||
table1.get(get).get();
|
||||
table2.get(get).get();
|
||||
|
||||
this.admin.disableTables(Pattern.compile("testDisableAndEnableTable.*")).join();
|
||||
this.admin.disableTables(Pattern.compile(tableName.getNameAsString() + ".*")).join();
|
||||
|
||||
// Test that tables are disabled
|
||||
get = new Get(row);
|
||||
get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
|
||||
get.addColumn(FAMILY, qualifier);
|
||||
boolean ok = false;
|
||||
try {
|
||||
ht1.get(get);
|
||||
ht2.get(get);
|
||||
} catch (org.apache.hadoop.hbase.DoNotRetryIOException e) {
|
||||
table1.get(get).get();
|
||||
} catch (ExecutionException e) {
|
||||
ok = true;
|
||||
}
|
||||
assertTrue(ok);
|
||||
|
||||
ok = false;
|
||||
try {
|
||||
table2.get(get).get();
|
||||
} catch (ExecutionException e) {
|
||||
ok = true;
|
||||
}
|
||||
assertTrue(ok);
|
||||
assertEquals(TableState.State.DISABLED, getStateFromMeta(tableName1));
|
||||
assertEquals(TableState.State.DISABLED, getStateFromMeta(tableName2));
|
||||
|
||||
assertTrue(ok);
|
||||
this.admin.enableTables(Pattern.compile("testDisableAndEnableTable.*")).join();
|
||||
this.admin.enableTables(Pattern.compile("testDisableAndEnableTables.*")).join();
|
||||
|
||||
// Test that tables are enabled
|
||||
try {
|
||||
ht1.get(get);
|
||||
} catch (IOException e) {
|
||||
table1.get(get).get();
|
||||
} catch (Exception e) {
|
||||
ok = false;
|
||||
}
|
||||
try {
|
||||
ht2.get(get);
|
||||
} catch (IOException e) {
|
||||
table2.get(get).get();
|
||||
} catch (Exception e) {
|
||||
ok = false;
|
||||
}
|
||||
assertTrue(ok);
|
||||
|
||||
ht1.close();
|
||||
ht2.close();
|
||||
|
||||
assertEquals(TableState.State.ENABLED, getStateFromMeta(tableName1));
|
||||
assertEquals(TableState.State.ENABLED, getStateFromMeta(tableName2));
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
@Test
|
||||
public void testEnableTableRetainAssignment() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
byte[][] splitKeys = { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 }, new byte[] { 3, 3, 3 },
|
||||
byte[][] splitKeys =
|
||||
{ new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 }, new byte[] { 3, 3, 3 },
|
||||
new byte[] { 4, 4, 4 }, new byte[] { 5, 5, 5 }, new byte[] { 6, 6, 6 },
|
||||
new byte[] { 7, 7, 7 }, new byte[] { 8, 8, 8 }, new byte[] { 9, 9, 9 } };
|
||||
int expectedRegions = splitKeys.length + 1;
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
|
||||
admin.createTable(desc, splitKeys).join();
|
||||
|
||||
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
|
||||
List<HRegionLocation> regions = l.getAllRegionLocations();
|
||||
createTableWithDefaultConf(tableName, Optional.of(splitKeys));
|
||||
|
||||
RawAsyncTable metaTable = ASYNC_CONN.getRawTable(META_TABLE_NAME);
|
||||
List<HRegionLocation> regions =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
assertEquals(
|
||||
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
|
||||
expectedRegions, regions.size());
|
||||
|
||||
// Disable table.
|
||||
admin.disableTable(tableName).join();
|
||||
// Enable table, use retain assignment to assign regions.
|
||||
admin.enableTable(tableName).join();
|
||||
List<HRegionLocation> regions2 = l.getAllRegionLocations();
|
||||
|
||||
List<HRegionLocation> regions2 =
|
||||
AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName)).get();
|
||||
// Check the assignment.
|
||||
assertEquals(regions.size(), regions2.size());
|
||||
assertTrue(regions2.containsAll(regions));
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
@Test
|
||||
public void testDisableCatalogTable() throws Exception {
|
||||
try {
|
||||
this.admin.disableTable(TableName.META_TABLE_NAME).join();
|
||||
|
@ -649,150 +606,116 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
}
|
||||
// Before the fix for HBASE-6146, the below table creation was failing as the hbase:meta table
|
||||
// actually getting disabled by the disableTable() call.
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName().getBytes()));
|
||||
HColumnDescriptor hcd = new HColumnDescriptor("cf1".getBytes());
|
||||
htd.addFamily(hcd);
|
||||
admin.createTable(htd).join();
|
||||
createTableWithDefaultConf(tableName);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddColumnFamily() throws IOException {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
public void testAddColumnFamily() throws Exception {
|
||||
// Create a table with two families
|
||||
HTableDescriptor baseHtd = new HTableDescriptor(tableName);
|
||||
baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
|
||||
admin.createTable(baseHtd).join();
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_0).build());
|
||||
admin.createTable(builder.build()).join();
|
||||
admin.disableTable(tableName).join();
|
||||
try {
|
||||
// Verify the table descriptor
|
||||
verifyTableDescriptor(tableName, FAMILY_0);
|
||||
|
||||
// Modify the table removing one family and verify the descriptor
|
||||
admin.addColumnFamily(tableName, new HColumnDescriptor(FAMILY_1)).join();
|
||||
admin.addColumnFamily(tableName, ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_1).build())
|
||||
.join();
|
||||
verifyTableDescriptor(tableName, FAMILY_0, FAMILY_1);
|
||||
} finally {
|
||||
admin.deleteTable(tableName);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddSameColumnFamilyTwice() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
// Create a table with one families
|
||||
HTableDescriptor baseHtd = new HTableDescriptor(tableName);
|
||||
baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
|
||||
admin.createTable(baseHtd).join();
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_0).build());
|
||||
admin.createTable(builder.build()).join();
|
||||
admin.disableTable(tableName).join();
|
||||
try {
|
||||
// Verify the table descriptor
|
||||
verifyTableDescriptor(tableName, FAMILY_0);
|
||||
|
||||
// Modify the table removing one family and verify the descriptor
|
||||
this.admin.addColumnFamily(tableName, new HColumnDescriptor(FAMILY_1)).join();
|
||||
admin.addColumnFamily(tableName, ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_1).build())
|
||||
.join();
|
||||
verifyTableDescriptor(tableName, FAMILY_0, FAMILY_1);
|
||||
|
||||
try {
|
||||
// Add same column family again - expect failure
|
||||
this.admin.addColumnFamily(tableName, new HColumnDescriptor(FAMILY_1)).join();
|
||||
this.admin.addColumnFamily(tableName,
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_1).build()).join();
|
||||
Assert.fail("Delete a non-exist column family should fail");
|
||||
} catch (Exception e) {
|
||||
// Expected.
|
||||
}
|
||||
} finally {
|
||||
admin.deleteTable(tableName).join();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testModifyColumnFamily() throws Exception {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
|
||||
HColumnDescriptor cfDescriptor = new HColumnDescriptor(FAMILY_0);
|
||||
int blockSize = cfDescriptor.getBlocksize();
|
||||
// Create a table with one families
|
||||
HTableDescriptor baseHtd = new HTableDescriptor(tableName);
|
||||
baseHtd.addFamily(cfDescriptor);
|
||||
admin.createTable(baseHtd).join();
|
||||
TableDescriptorBuilder tdBuilder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_0).build();
|
||||
int blockSize = cfd.getBlocksize();
|
||||
admin.createTable(tdBuilder.addColumnFamily(cfd).build()).join();
|
||||
admin.disableTable(tableName).join();
|
||||
try {
|
||||
// Verify the table descriptor
|
||||
verifyTableDescriptor(tableName, FAMILY_0);
|
||||
|
||||
int newBlockSize = 2 * blockSize;
|
||||
cfDescriptor.setBlocksize(newBlockSize);
|
||||
|
||||
cfd = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_0).setBlocksize(newBlockSize).build();
|
||||
// Modify colymn family
|
||||
admin.modifyColumnFamily(tableName, cfDescriptor).join();
|
||||
admin.modifyColumnFamily(tableName, cfd).join();
|
||||
|
||||
TableDescriptor htd = admin.getTableDescriptor(tableName).get();
|
||||
ColumnFamilyDescriptor hcfd = htd.getColumnFamily(FAMILY_0);
|
||||
assertTrue(hcfd.getBlocksize() == newBlockSize);
|
||||
} finally {
|
||||
admin.deleteTable(tableName).join();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testModifyNonExistingColumnFamily() throws IOException {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
|
||||
HColumnDescriptor cfDescriptor = new HColumnDescriptor(FAMILY_1);
|
||||
int blockSize = cfDescriptor.getBlocksize();
|
||||
// Create a table with one families
|
||||
HTableDescriptor baseHtd = new HTableDescriptor(tableName);
|
||||
baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
|
||||
admin.createTable(baseHtd).join();
|
||||
public void testModifyNonExistingColumnFamily() throws Exception {
|
||||
TableDescriptorBuilder tdBuilder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_0).build();
|
||||
int blockSize = cfd.getBlocksize();
|
||||
admin.createTable(tdBuilder.addColumnFamily(cfd).build()).join();
|
||||
admin.disableTable(tableName).join();
|
||||
try {
|
||||
// Verify the table descriptor
|
||||
verifyTableDescriptor(tableName, FAMILY_0);
|
||||
|
||||
int newBlockSize = 2 * blockSize;
|
||||
cfDescriptor.setBlocksize(newBlockSize);
|
||||
cfd = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_1).setBlocksize(newBlockSize).build();
|
||||
|
||||
// Modify a column family that is not in the table.
|
||||
try {
|
||||
admin.modifyColumnFamily(tableName, cfDescriptor).join();
|
||||
admin.modifyColumnFamily(tableName, cfd).join();
|
||||
Assert.fail("Modify a non-exist column family should fail");
|
||||
} catch (Exception e) {
|
||||
// Expected.
|
||||
}
|
||||
} finally {
|
||||
admin.deleteTable(tableName).join();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeleteColumnFamily() throws IOException {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
public void testDeleteColumnFamily() throws Exception {
|
||||
// Create a table with two families
|
||||
HTableDescriptor baseHtd = new HTableDescriptor(tableName);
|
||||
baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
|
||||
baseHtd.addFamily(new HColumnDescriptor(FAMILY_1));
|
||||
admin.createTable(baseHtd).join();
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_0).build())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_1).build());
|
||||
admin.createTable(builder.build()).join();
|
||||
admin.disableTable(tableName).join();
|
||||
try {
|
||||
// Verify the table descriptor
|
||||
verifyTableDescriptor(tableName, FAMILY_0, FAMILY_1);
|
||||
|
||||
// Modify the table removing one family and verify the descriptor
|
||||
admin.deleteColumnFamily(tableName, FAMILY_1).join();
|
||||
verifyTableDescriptor(tableName, FAMILY_0);
|
||||
} finally {
|
||||
admin.deleteTable(tableName).join();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeleteSameColumnFamilyTwice() throws IOException {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
public void testDeleteSameColumnFamilyTwice() throws Exception {
|
||||
// Create a table with two families
|
||||
HTableDescriptor baseHtd = new HTableDescriptor(tableName);
|
||||
baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
|
||||
baseHtd.addFamily(new HColumnDescriptor(FAMILY_1));
|
||||
admin.createTable(baseHtd).join();
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_0).build())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_1).build());
|
||||
admin.createTable(builder.build()).join();
|
||||
admin.disableTable(tableName).join();
|
||||
try {
|
||||
// Verify the table descriptor
|
||||
verifyTableDescriptor(tableName, FAMILY_0, FAMILY_1);
|
||||
|
||||
|
@ -807,30 +730,25 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
} catch (Exception e) {
|
||||
// Expected.
|
||||
}
|
||||
} finally {
|
||||
admin.deleteTable(tableName).join();
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyTableDescriptor(final TableName tableName, final byte[]... families)
|
||||
throws IOException {
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
|
||||
throws Exception {
|
||||
// Verify descriptor from master
|
||||
HTableDescriptor htd = admin.getTableDescriptor(tableName);
|
||||
TableDescriptor htd = admin.getTableDescriptor(tableName).get();
|
||||
verifyTableDescriptor(htd, tableName, families);
|
||||
|
||||
// Verify descriptor from HDFS
|
||||
MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
|
||||
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
|
||||
HTableDescriptor td = FSTableDescriptors
|
||||
.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||
HTableDescriptor td =
|
||||
FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
|
||||
verifyTableDescriptor(td, tableName, families);
|
||||
}
|
||||
|
||||
private void verifyTableDescriptor(final HTableDescriptor htd, final TableName tableName,
|
||||
private void verifyTableDescriptor(final TableDescriptor htd, final TableName tableName,
|
||||
final byte[]... families) {
|
||||
Set<byte[]> htdFamilies = htd.getFamiliesKeys();
|
||||
Set<byte[]> htdFamilies = htd.getColumnFamilyNames();
|
||||
assertEquals(tableName, htd.getTableName());
|
||||
assertEquals(families.length, htdFamilies.size());
|
||||
for (byte[] familyName : families) {
|
||||
|
@ -840,28 +758,20 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
|
|||
|
||||
@Test
|
||||
public void testIsTableEnabledAndDisabled() throws Exception {
|
||||
final TableName table = TableName.valueOf("testIsTableEnabledAndDisabled");
|
||||
HTableDescriptor desc = new HTableDescriptor(table);
|
||||
desc.addFamily(new HColumnDescriptor(FAMILY));
|
||||
admin.createTable(desc).join();
|
||||
assertTrue(admin.isTableEnabled(table).get());
|
||||
assertFalse(admin.isTableDisabled(table).get());
|
||||
admin.disableTable(table).join();
|
||||
assertFalse(admin.isTableEnabled(table).get());
|
||||
assertTrue(admin.isTableDisabled(table).get());
|
||||
admin.deleteTable(table).join();
|
||||
createTableWithDefaultConf(tableName);
|
||||
assertTrue(admin.isTableEnabled(tableName).get());
|
||||
assertFalse(admin.isTableDisabled(tableName).get());
|
||||
admin.disableTable(tableName).join();
|
||||
assertFalse(admin.isTableEnabled(tableName).get());
|
||||
assertTrue(admin.isTableDisabled(tableName).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTableAvailableWithRandomSplitKeys() throws Exception {
|
||||
TableName tableName = TableName.valueOf("testTableAvailableWithRandomSplitKeys");
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.addFamily(new HColumnDescriptor("col"));
|
||||
createTableWithDefaultConf(tableName);
|
||||
byte[][] splitKeys = new byte[1][];
|
||||
splitKeys = new byte[][] { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 } };
|
||||
admin.createTable(desc).join();
|
||||
boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys).get();
|
||||
assertFalse("Table should be created with 1 row in META", tableAvailable);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue