HBASE-22002 Remove the deprecated methods in Admin interface

Signed-off-by: Zheng Hu <openinx@gmail.com>
This commit is contained in:
Duo Zhang 2019-03-13 17:21:43 +08:00 committed by zhangduo
parent e536e6a2aa
commit 3c5761e365
122 changed files with 911 additions and 2657 deletions

View File

@ -66,9 +66,8 @@ public final class HelloHBase {
* CLASSPATH, to enable creation of Connection to HBase via ZooKeeper.
*/
try (Connection connection = ConnectionFactory.createConnection();
Admin admin = connection.getAdmin()) {
admin.getClusterStatus(); // assure connection successfully established
Admin admin = connection.getAdmin()) {
admin.getClusterMetrics(); // assure connection successfully established
System.out.println("\n*** Hello HBase! -- Connection has been "
+ "established via ZooKeeper!!\n");

View File

@ -65,9 +65,8 @@ public final class HelloHBase {
* CLASSPATH, to enable creation of Connection to HBase via ZooKeeper.
*/
try (Connection connection = ConnectionFactory.createConnection();
Admin admin = connection.getAdmin()) {
admin.getClusterStatus(); // assure connection successfully established
Admin admin = connection.getAdmin()) {
admin.getClusterMetrics(); // assure connection successfully established
System.out.println("\n*** Hello HBase! -- Connection has been "
+ "established via ZooKeeper!!\n");

View File

@ -26,11 +26,9 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.BackupHFileCleaner;
import org.apache.hadoop.hbase.backup.BackupInfo;
@ -45,15 +43,17 @@ import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.procedure.ProcedureManagerHost;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* Handles backup requests, creates backup info records in backup system table to keep track of
* backup sessions, dispatches backup request.
@ -208,9 +208,9 @@ public class BackupManager implements Closeable {
if (type == BackupType.FULL && (tableList == null || tableList.isEmpty())) {
// If table list is null for full backup, which means backup all tables. Then fill the table
// list with all user tables from meta. It no table available, throw the request exception.
HTableDescriptor[] htds = null;
List<TableDescriptor> htds = null;
try (Admin admin = conn.getAdmin()) {
htds = admin.listTables();
htds = admin.listTableDescriptors();
} catch (Exception e) {
throw new BackupException(e);
}
@ -219,7 +219,7 @@ public class BackupManager implements Closeable {
throw new BackupException("No table exists for full backup of all tables.");
} else {
tableList = new ArrayList<>();
for (HTableDescriptor hTableDescriptor : htds) {
for (TableDescriptor hTableDescriptor : htds) {
TableName tn = hTableDescriptor.getTableName();
if (tn.equals(BackupSystemTable.getTableName(conf))) {
// skip backup system table

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.backup;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@ -134,8 +133,8 @@ public class TestIncrementalBackup extends TestBackupBase {
long startSplitTime = EnvironmentEdgeManager.currentTime();
try {
admin.splitRegion(name);
} catch (IOException e) {
admin.splitRegionAsync(name, null).get();
} catch (Exception e) {
// although split fail, this may not affect following check in current API,
// exception will be thrown.
LOG.debug("region is not splittable, because " + e);

View File

@ -45,7 +45,6 @@ import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.CacheEvictionStats;
import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
import org.apache.hadoop.hbase.ClusterMetrics;
@ -54,9 +53,7 @@ import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
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.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@ -136,7 +133,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Coprocesso
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
@ -170,8 +166,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedu
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
@ -461,50 +455,14 @@ public class HBaseAdmin implements Admin {
});
}
@Override
public HTableDescriptor[] listTables() throws IOException {
return listTables((Pattern)null, false);
}
@Override
public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
return listTables(pattern, false);
}
@Override
public HTableDescriptor[] listTables(String regex) throws IOException {
return listTables(Pattern.compile(regex), false);
}
@Override
public HTableDescriptor[] listTables(final Pattern pattern, final boolean includeSysTables)
throws IOException {
return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
getRpcControllerFactory()) {
@Override
protected HTableDescriptor[] rpcCall() throws Exception {
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(),
req)).stream().map(ImmutableHTableDescriptor::new).toArray(HTableDescriptor[]::new);
}
});
}
@Override
public HTableDescriptor[] listTables(String regex, boolean includeSysTables)
throws IOException {
return listTables(Pattern.compile(regex), includeSysTables);
}
@Override
public TableName[] listTableNames() throws IOException {
return listTableNames((Pattern)null, false);
}
@Override
public TableName[] listTableNames(String regex) throws IOException {
return listTableNames(Pattern.compile(regex), false);
public TableName[] listTableNames(Pattern pattern) throws IOException {
return listTableNames(pattern, false);
}
@Override
@ -522,18 +480,6 @@ public class HBaseAdmin implements Admin {
});
}
@Override
public TableName[] listTableNames(final String regex, final boolean includeSysTables)
throws IOException {
return listTableNames(Pattern.compile(regex), includeSysTables);
}
@Override
public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
return getHTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
operationTimeout, rpcTimeout);
}
static TableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
int operationTimeout, int rpcTimeout) throws IOException {
@ -557,38 +503,6 @@ public class HBaseAdmin implements Admin {
throw new TableNotFoundException(tableName.getNameAsString());
}
/**
* @deprecated since 2.0 version and will be removed in 3.0 version.
* use {@link #getTableDescriptor(TableName,
* Connection, RpcRetryingCallerFactory,RpcControllerFactory,int,int)}
*/
@Deprecated
static HTableDescriptor getHTableDescriptor(final TableName tableName, Connection connection,
RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
int operationTimeout, int rpcTimeout) throws IOException {
if (tableName == null) {
return null;
}
HTableDescriptor htd =
executeCallable(new MasterCallable<HTableDescriptor>(connection, rpcControllerFactory) {
@Override
protected HTableDescriptor rpcCall() throws Exception {
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(tableName);
GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
if (!htds.getTableSchemaList().isEmpty()) {
return new ImmutableHTableDescriptor(
ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0)));
}
return null;
}
}, rpcCallerFactory, operationTimeout, rpcTimeout);
if (htd != null) {
return new ImmutableHTableDescriptor(htd);
}
throw new TableNotFoundException(tableName.getNameAsString());
}
private long getPauseTime(int tries) {
int triesCount = tries;
if (triesCount >= HConstants.RETRY_BACKOFF.length) {
@ -673,7 +587,7 @@ public class HBaseAdmin implements Admin {
}
@Override
protected TableDescriptor getTableDescriptor() {
protected TableDescriptor getDescriptor() {
return desc;
}
@ -735,36 +649,6 @@ public class HBaseAdmin implements Admin {
}
}
@Override
public HTableDescriptor[] deleteTables(String regex) throws IOException {
return deleteTables(Pattern.compile(regex));
}
/**
* Delete tables matching the passed in pattern and wait on completion.
*
* Warning: Use this method carefully, there is no prompting and the effect is
* immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
* {@link #deleteTable(TableName)}
*
* @param pattern The pattern to match table names against
* @return Table descriptors for tables that couldn't be deleted
* @throws IOException
*/
@Override
public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
List<HTableDescriptor> failed = new LinkedList<>();
for (HTableDescriptor table : listTables(pattern)) {
try {
deleteTable(table.getTableName());
} catch (IOException ex) {
LOG.info("Failed to delete table " + table.getTableName(), ex);
failed.add(table);
}
}
return failed.toArray(new HTableDescriptor[failed.size()]);
}
@Override
public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
throws IOException {
@ -863,27 +747,6 @@ public class HBaseAdmin implements Admin {
}
}
@Override
public HTableDescriptor[] enableTables(String regex) throws IOException {
return enableTables(Pattern.compile(regex));
}
@Override
public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
List<HTableDescriptor> failed = new LinkedList<>();
for (HTableDescriptor table : listTables(pattern)) {
if (isTableDisabled(table.getTableName())) {
try {
enableTable(table.getTableName());
} catch (IOException ex) {
LOG.info("Failed to enable table " + table.getTableName(), ex);
failed.add(table);
}
}
}
return failed.toArray(new HTableDescriptor[failed.size()]);
}
@Override
public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
TableName.isLegalFullyQualifiedTableName(tableName.getName());
@ -923,27 +786,6 @@ public class HBaseAdmin implements Admin {
}
}
@Override
public HTableDescriptor[] disableTables(String regex) throws IOException {
return disableTables(Pattern.compile(regex));
}
@Override
public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
List<HTableDescriptor> failed = new LinkedList<>();
for (HTableDescriptor table : listTables(pattern)) {
if (isTableEnabled(table.getTableName())) {
try {
disableTable(table.getTableName());
} catch (IOException ex) {
LOG.info("Failed to disable table " + table.getTableName(), ex);
failed.add(table);
}
}
}
return failed.toArray(new HTableDescriptor[failed.size()]);
}
@Override
public boolean isTableEnabled(final TableName tableName) throws IOException {
checkTableExists(tableName);
@ -970,33 +812,6 @@ public class HBaseAdmin implements Admin {
return connection.isTableAvailable(tableName, null);
}
@Override
public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException {
return connection.isTableAvailable(tableName, splitKeys);
}
@Override
public Pair<Integer, Integer> getAlterStatus(final TableName tableName) throws IOException {
return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection(),
getRpcControllerFactory()) {
@Override
protected Pair<Integer, Integer> rpcCall() throws Exception {
setPriority(tableName);
GetSchemaAlterStatusRequest req = RequestConverter
.buildGetSchemaAlterStatusRequest(tableName);
GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(getRpcController(), req);
Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
ret.getTotalRegions());
return pair;
}
});
}
@Override
public Pair<Integer, Integer> getAlterStatus(final byte[] tableName) throws IOException {
return getAlterStatus(TableName.valueOf(tableName));
}
@Override
public Future<Void> addColumnFamilyAsync(final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException {
@ -1029,18 +844,6 @@ public class HBaseAdmin implements Admin {
}
}
/**
* {@inheritDoc}
* @deprecated Since 2.0. Will be removed in 3.0. Use
* {@link #deleteColumnFamily(TableName, byte[])} instead.
*/
@Override
@Deprecated
public void deleteColumn(final TableName tableName, final byte[] columnFamily)
throws IOException {
deleteColumnFamily(tableName, columnFamily);
}
@Override
public Future<Void> deleteColumnFamilyAsync(final TableName tableName, final byte[] columnFamily)
throws IOException {
@ -1107,45 +910,6 @@ public class HBaseAdmin implements Admin {
}
}
@Deprecated
@Override
public void closeRegion(final String regionName, final String unused) throws IOException {
unassign(Bytes.toBytes(regionName), true);
}
@Deprecated
@Override
public void closeRegion(final byte [] regionName, final String unused) throws IOException {
unassign(regionName, true);
}
@Deprecated
@Override
public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
final String unused) throws IOException {
unassign(Bytes.toBytes(encodedRegionName), true);
return true;
}
@Deprecated
@Override
public void closeRegion(final ServerName unused, final HRegionInfo hri) throws IOException {
unassign(hri.getRegionName(), true);
}
/**
* @param sn
* @return List of {@link HRegionInfo}.
* @throws IOException
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegions(ServerName)}.
*/
@Deprecated
@Override
public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
return getRegions(sn).stream().map(ImmutableHRegionInfo::new).collect(Collectors.toList());
}
@Override
public void flush(final TableName tableName) throws IOException {
checkTableExists(tableName);
@ -1659,24 +1423,6 @@ public class HBaseAdmin implements Admin {
TimeUnit.MILLISECONDS);
}
/**
* Merge two regions. Asynchronous operation.
* @param nameOfRegionA encoded or full name of region a
* @param nameOfRegionB encoded or full name of region b
* @param forcible true if do a compulsory merge, otherwise we will only merge
* two adjacent regions
* @throws IOException
* @deprecated Since 2.0. Will be removed in 3.0. Use
* {@link #mergeRegionsAsync(byte[], byte[], boolean)} instead.
*/
@Deprecated
@Override
public void mergeRegions(final byte[] nameOfRegionA,
final byte[] nameOfRegionB, final boolean forcible)
throws IOException {
mergeRegionsAsync(nameOfRegionA, nameOfRegionB, forcible);
}
/**
* Merge two regions. Asynchronous operation.
* @param nameOfRegionA encoded or full name of region a
@ -1710,9 +1456,9 @@ public class HBaseAdmin implements Admin {
assert(nameofRegionsToMerge.length >= 2);
byte[][] encodedNameofRegionsToMerge = new byte[nameofRegionsToMerge.length][];
for(int i = 0; i < nameofRegionsToMerge.length; i++) {
encodedNameofRegionsToMerge[i] = HRegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ?
encodedNameofRegionsToMerge[i] = RegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ?
nameofRegionsToMerge[i] :
Bytes.toBytes(HRegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
Bytes.toBytes(RegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
}
TableName tableName = null;
@ -1722,7 +1468,7 @@ public class HBaseAdmin implements Admin {
pair = getRegion(nameofRegionsToMerge[i]);
if (pair != null) {
if (pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
if (pair.getFirst().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
throw new IllegalArgumentException ("Can't invoke merge on non-default regions directly");
}
if (tableName == null) {
@ -1811,18 +1557,17 @@ public class HBaseAdmin implements Admin {
@Override
public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint)
throws IOException {
byte[] encodedNameofRegionToSplit = HRegionInfo.isEncodedRegionName(regionName) ?
regionName : Bytes.toBytes(HRegionInfo.encodeRegionName(regionName));
byte[] encodedNameofRegionToSplit = RegionInfo.isEncodedRegionName(regionName) ?
regionName : Bytes.toBytes(RegionInfo.encodeRegionName(regionName));
Pair<RegionInfo, ServerName> pair = getRegion(regionName);
if (pair != null) {
if (pair.getFirst() != null &&
pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
pair.getFirst().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
throw new IllegalArgumentException ("Can't invoke split on non-default regions directly");
}
} else {
throw new UnknownRegionException (
"Can't invoke merge on unknown region "
+ Bytes.toStringBinary(encodedNameofRegionToSplit));
throw new UnknownRegionException(
"Can't invoke split on unknown region " + Bytes.toStringBinary(encodedNameofRegionToSplit));
}
return splitRegionAsync(pair.getFirst(), splitPoint);
@ -1876,11 +1621,6 @@ public class HBaseAdmin implements Admin {
split(tableName, null);
}
@Override
public void splitRegion(final byte[] regionName) throws IOException {
splitRegion(regionName, null);
}
@Override
public void split(final TableName tableName, final byte[] splitPoint) throws IOException {
checkTableExists(tableName);
@ -1904,23 +1644,6 @@ public class HBaseAdmin implements Admin {
}
}
@Override
public void splitRegion(final byte[] regionName, final byte [] splitPoint) throws IOException {
Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
}
if (regionServerPair.getFirst() != null &&
regionServerPair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
throw new IllegalArgumentException("Can't split replicas directly. "
+ "Replicas are auto-split when their primary is split.");
}
if (regionServerPair.getSecond() == null) {
throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
}
splitRegionAsync(regionServerPair.getFirst(), splitPoint);
}
private static class ModifyTableFuture extends TableFuture<Void> {
public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName,
final ModifyTableResponse response) {
@ -1936,16 +1659,6 @@ public class HBaseAdmin implements Admin {
public String getOperationType() {
return "MODIFY";
}
@Override
protected Void postOperationResult(final Void result, final long deadlineTs)
throws IOException, TimeoutException {
// The modify operation on the table is asynchronous on the server side irrespective
// of whether Procedure V2 is supported or not. So, we wait in the client till
// all regions get updated.
waitForSchemaUpdate(deadlineTs);
return result;
}
}
/**
@ -1978,9 +1691,9 @@ public class HBaseAdmin implements Admin {
ServerName sn = null;
if (rl != null) {
for (HRegionLocation h : rl.getRegionLocations()) {
if (h != null && encodedName.equals(h.getRegionInfo().getEncodedName())) {
if (h != null && encodedName.equals(h.getRegion().getEncodedName())) {
sn = h.getServerName();
info = h.getRegionInfo();
info = h.getRegion();
matched = true;
}
}
@ -2005,13 +1718,12 @@ public class HBaseAdmin implements Admin {
* name, the input is returned as is. We don't throw unknown
* region exception.
*/
private byte[] getRegionName(
final byte[] regionNameOrEncodedRegionName) throws IOException {
private byte[] getRegionName(final byte[] regionNameOrEncodedRegionName) throws IOException {
if (Bytes.equals(regionNameOrEncodedRegionName,
HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
|| Bytes.equals(regionNameOrEncodedRegionName,
HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) ||
Bytes.equals(regionNameOrEncodedRegionName,
RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
return RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName();
}
byte[] tmp = regionNameOrEncodedRegionName;
Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
@ -2272,25 +1984,6 @@ public class HBaseAdmin implements Admin {
});
}
@Override
public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
getRpcControllerFactory()) {
@Override
protected HTableDescriptor[] rpcCall() throws Exception {
List<TableSchema> list =
master.listTableDescriptorsByNamespace(getRpcController(),
ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
.build()).getTableSchemaList();
HTableDescriptor[] res = new HTableDescriptor[list.size()];
for(int i=0; i < list.size(); i++) {
res[i] = new ImmutableHTableDescriptor(ProtobufUtil.toTableDescriptor(list.get(i)));
}
return res;
}
});
}
@Override
public TableName[] listTableNamesByNamespace(final String name) throws IOException {
return executeCallable(new MasterCallable<TableName[]>(getConnection(),
@ -2312,10 +2005,11 @@ public class HBaseAdmin implements Admin {
/**
* Is HBase available? Throw an exception if not.
* <p/>
* TODO: do not expose ZKConnectionException.
* @param conf system configuration
* @throws MasterNotRunningException if the master is not running.
* @throws ZooKeeperConnectionException if unable to connect to zookeeper. // TODO do not expose
* ZKConnectionException.
* @throws ZooKeeperConnectionException if unable to connect to zookeeper.
*/
public static void available(final Configuration conf)
throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
@ -2327,59 +2021,12 @@ public class HBaseAdmin implements Admin {
// Check ZK first.
// If the connection exists, we may have a connection to ZK that does not work anymore
try (ClusterConnection connection =
(ClusterConnection) ConnectionFactory.createConnection(copyOfConf)) {
(ClusterConnection) ConnectionFactory.createConnection(copyOfConf)) {
// can throw MasterNotRunningException
connection.isMasterRunning();
}
}
/**
*
* @param tableName
* @return List of {@link HRegionInfo}.
* @throws IOException
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getRegions(TableName)}.
*/
@Deprecated
@Override
public List<HRegionInfo> getTableRegions(final TableName tableName)
throws IOException {
return getRegions(tableName).stream()
.map(ImmutableHRegionInfo::new)
.collect(Collectors.toList());
}
@Override
public synchronized void close() throws IOException {
}
@Override
public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
throws IOException {
return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
getRpcControllerFactory()) {
@Override
protected HTableDescriptor[] rpcCall() throws Exception {
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(tableNames);
return ProtobufUtil
.toTableDescriptorList(master.getTableDescriptors(getRpcController(), req)).stream()
.map(ImmutableHTableDescriptor::new).toArray(HTableDescriptor[]::new);
}
});
}
@Override
public HTableDescriptor[] getTableDescriptors(List<String> names)
throws IOException {
List<TableName> tableNames = new ArrayList<>(names.size());
for(String name : names) {
tableNames.add(TableName.valueOf(name));
}
return getTableDescriptorsByTableName(tableNames);
}
private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
FailedLogCloseException {
final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
@ -2539,12 +2186,6 @@ public class HBaseAdmin implements Admin {
}).getDone();
}
@Override
public void restoreSnapshot(final byte[] snapshotName)
throws IOException, RestoreSnapshotException {
restoreSnapshot(Bytes.toString(snapshotName));
}
@Override
public void restoreSnapshot(final String snapshotName)
throws IOException, RestoreSnapshotException {
@ -2554,12 +2195,6 @@ public class HBaseAdmin implements Admin {
restoreSnapshot(snapshotName, takeFailSafeSnapshot);
}
@Override
public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot)
throws IOException, RestoreSnapshotException {
restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
}
/**
* Check whether the snapshot exists and contains disabled table
*
@ -2659,24 +2294,6 @@ public class HBaseAdmin implements Admin {
}
}
@Override
public Future<Void> restoreSnapshotAsync(final String snapshotName)
throws IOException, RestoreSnapshotException {
TableName tableName = getTableNameBeforeRestoreSnapshot(snapshotName);
// The table does not exists, switch to clone.
if (!tableExists(tableName)) {
return cloneSnapshotAsync(snapshotName, tableName);
}
// Check if the table is disabled
if (!isTableDisabled(tableName)) {
throw new TableNotDisabledException(tableName);
}
return internalRestoreSnapshotAsync(snapshotName, tableName, false);
}
@Override
public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
@ -2846,11 +2463,6 @@ public class HBaseAdmin implements Admin {
});
}
@Override
public List<SnapshotDescription> listSnapshots(String regex) throws IOException {
return listSnapshots(Pattern.compile(regex));
}
@Override
public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
List<SnapshotDescription> matched = new LinkedList<>();
@ -2863,12 +2475,6 @@ public class HBaseAdmin implements Admin {
return matched;
}
@Override
public List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
String snapshotNameRegex) throws IOException {
return listTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
}
@Override
public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern) throws IOException {
@ -2909,11 +2515,6 @@ public class HBaseAdmin implements Admin {
});
}
@Override
public void deleteSnapshots(final String regex) throws IOException {
deleteSnapshots(Pattern.compile(regex));
}
@Override
public void deleteSnapshots(final Pattern pattern) throws IOException {
List<SnapshotDescription> snapshots = listSnapshots(pattern);
@ -2938,12 +2539,6 @@ public class HBaseAdmin implements Admin {
});
}
@Override
public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex)
throws IOException {
deleteTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
}
@Override
public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
throws IOException {
@ -2969,11 +2564,6 @@ public class HBaseAdmin implements Admin {
});
}
@Override
public QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException {
return QuotaRetriever.open(conf, filter);
}
@Override
public List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException {
List<QuotaSettings> quotas = new LinkedList<>();
@ -3035,21 +2625,6 @@ public class HBaseAdmin implements Admin {
};
}
/**
* Simple {@link Abortable}, throwing RuntimeException on abort.
*/
private static class ThrowableAbortable implements Abortable {
@Override
public void abort(String why, Throwable e) {
throw new RuntimeException(why, e);
}
@Override
public boolean isAborted() {
return true;
}
}
@Override
public CoprocessorRpcChannel coprocessorService(final ServerName serverName) {
return new SyncCoprocessorRpcChannel() {
@ -3560,7 +3135,7 @@ public class HBaseAdmin implements Admin {
/**
* @return the table descriptor
*/
protected TableDescriptor getTableDescriptor() throws IOException {
protected TableDescriptor getDescriptor() throws IOException {
return getAdmin().getDescriptor(getTableName());
}
@ -3642,19 +3217,9 @@ public class HBaseAdmin implements Admin {
});
}
protected void waitForSchemaUpdate(final long deadlineTs)
throws IOException, TimeoutException {
waitForState(deadlineTs, new TableWaitForStateCallable() {
@Override
public boolean checkState(int tries) throws IOException {
return getAdmin().getAlterStatus(tableName).getFirst() == 0;
}
});
}
protected void waitForAllRegionsOnline(final long deadlineTs, final byte[][] splitKeys)
throws IOException, TimeoutException {
final TableDescriptor desc = getTableDescriptor();
final TableDescriptor desc = getDescriptor();
final AtomicInteger actualRegCount = new AtomicInteger(0);
final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
@Override
@ -3668,16 +3233,18 @@ public class HBaseAdmin implements Admin {
if (l == null) {
return true;
}
if (!l.getRegionInfo().getTable().equals(desc.getTableName())) {
if (!l.getRegion().getTable().equals(desc.getTableName())) {
return false;
}
if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
if (l.getRegion().isOffline() || l.getRegion().isSplit()) {
return true;
}
HRegionLocation[] locations = list.getRegionLocations();
for (HRegionLocation location : locations) {
if (location == null) continue;
ServerName serverName = location.getServerName();
// Make sure that regions are assigned to server
if (serverName != null && serverName.getHostAndPort() != null) {
if (serverName != null && serverName.getAddress() != null) {
actualRegCount.incrementAndGet();
}
}
@ -4132,7 +3699,7 @@ public class HBaseAdmin implements Admin {
@Override
protected List<ServerName> rpcCall() throws Exception {
ClearDeadServersRequest req = RequestConverter.
buildClearDeadServersRequest(servers == null? Collections.EMPTY_LIST: servers);
buildClearDeadServersRequest(servers == null? Collections.emptyList(): servers);
return ProtobufUtil.toServerNameList(
master.clearDeadServers(getRpcController(), req).getServerNameList());
}
@ -4146,7 +3713,7 @@ public class HBaseAdmin implements Admin {
if (tableExists(newTableName)) {
throw new TableExistsException(newTableName);
}
TableDescriptor htd = TableDescriptorBuilder.copy(newTableName, getTableDescriptor(tableName));
TableDescriptor htd = TableDescriptorBuilder.copy(newTableName, getDescriptor(tableName));
if (preserveSplits) {
createTable(htd, getTableSplits(tableName));
} else {
@ -4296,4 +3863,8 @@ public class HBaseAdmin implements Admin {
}
});
}
@Override
public void close() {
}
}

View File

@ -18,6 +18,8 @@
*/
package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
// DO NOT MAKE USE OF THESE IMPORTS! THEY ARE HERE FOR COPROCESSOR ENDPOINTS ONLY.
// Internally, we use shaded protobuf. This below are part of our public API.
//SEE ABOVE NOTE!
@ -25,40 +27,6 @@ import com.google.protobuf.Descriptors;
import com.google.protobuf.Message;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
@ -73,8 +41,39 @@ import java.util.concurrent.Future;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
/**
* An implementation of {@link Table}. Used to communicate with a single HBase table.
@ -223,17 +222,6 @@ public class HTable implements Table {
return this.connection;
}
@Override
@Deprecated
public HTableDescriptor getTableDescriptor() throws IOException {
HTableDescriptor htd = HBaseAdmin.getHTableDescriptor(tableName, connection, rpcCallerFactory,
rpcControllerFactory, operationTimeoutMs, readRpcTimeoutMs);
if (htd != null) {
return new ImmutableHTableDescriptor(htd);
}
return null;
}
@Override
public TableDescriptor getDescriptor() throws IOException {
return HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory,

View File

@ -21,23 +21,22 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.regex.Pattern;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.BlockingInterface;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Utility client for doing access control admin operations.
@ -254,7 +253,7 @@ public class AccessControlClient {
CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);
BlockingInterface protocol =
AccessControlProtos.AccessControlService.newBlockingStub(service);
HTableDescriptor[] htds = null;
List<TableDescriptor> htds = null;
if (tableRegex == null || tableRegex.isEmpty()) {
permList = AccessControlUtil.getUserPermissions(null, protocol, userName);
} else if (tableRegex.charAt(0) == '@') { // Namespaces
@ -268,8 +267,8 @@ public class AccessControlClient {
}
}
} else { // Tables
htds = admin.listTables(Pattern.compile(tableRegex), true);
for (HTableDescriptor htd : htds) {
htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
for (TableDescriptor htd : htds) {
permList.addAll(AccessControlUtil.getUserPermissions(null, protocol, htd.getTableName(),
null, null, userName));
}
@ -352,9 +351,9 @@ public class AccessControlClient {
CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);
BlockingInterface protocol =
AccessControlProtos.AccessControlService.newBlockingStub(service);
HTableDescriptor[] htds = admin.listTables(Pattern.compile(tableRegex), true);
List<TableDescriptor> htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
// Retrieve table permissions
for (HTableDescriptor htd : htds) {
for (TableDescriptor htd : htds) {
permList.addAll(AccessControlUtil.getUserPermissions(null, protocol, htd.getTableName(),
columnFamily, columnQualifier, userName));
}

View File

@ -139,7 +139,7 @@ public final class FutureUtils {
}
/**
* A helper class for getting the result of a Future, and convert the error to an
* A helper class for getting the result of a Future with timeout, and convert the error to an
* {@link IOException}.
*/
public static <T> T get(Future<T> future, long timeout, TimeUnit unit) throws IOException {

View File

@ -151,7 +151,7 @@ public class TestCoprocessorTableEndpoint {
private static void updateTable(HTableDescriptor desc) throws Exception {
Admin admin = TEST_UTIL.getAdmin();
admin.disableTable(desc.getTableName());
admin.modifyTable(desc.getTableName(), desc);
admin.modifyTable(desc);
admin.enableTable(desc.getTableName());
}

View File

@ -522,11 +522,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
// and the restore put the cluster back to Initial configuration, HAdmin instance will need
// to refresh its connections (otherwise it will return incorrect information) or we can
// point it to new instance.
try {
admin.close();
} catch (IOException ioe) {
LOG.warn("While closing the old connection", ioe);
}
admin.close();
this.admin = this.connection.getAdmin();
LOG.info("Added new HBaseAdmin");
return true;

View File

@ -24,7 +24,7 @@ import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.regex.Pattern;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.conf.Configuration;
@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@ -151,8 +152,10 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
public void cleanUpCluster() throws Exception {
if (!keepObjectsAtTheEnd) {
Admin admin = util.getAdmin();
admin.disableTables("ittable-\\d+");
admin.deleteTables("ittable-\\d+");
for (TableName tableName: admin.listTableNames(Pattern.compile("ittable-\\d+"))) {
admin.disableTable(tableName);
admin.deleteTable(tableName);
}
NamespaceDescriptor [] nsds = admin.listNamespaceDescriptors();
for(NamespaceDescriptor nsd: nsds) {
if(nsd.getName().matches("itnamespace\\d+")) {
@ -713,7 +716,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
admin.modifyTable(td);
// assertion
TableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
TableDescriptor freshTableDesc = admin.getDescriptor(tableName);
ColumnFamilyDescriptor freshColumnDesc = freshTableDesc.getColumnFamily(columnDesc.getName());
Assert.assertEquals("Encoding of column family: " + columnDesc + " was not altered",
freshColumnDesc.getDataBlockEncoding().getId(), id);
@ -781,7 +784,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
Admin admin = connection.getAdmin();
TableName tableName = selected.getTableName();
try (Table table = connection.getTable(tableName)){
ArrayList<HRegionInfo> regionInfos = new ArrayList<>(admin.getTableRegions(
ArrayList<RegionInfo> regionInfos = new ArrayList<>(admin.getRegions(
selected.getTableName()));
int numRegions = regionInfos.size();
// average number of rows to be added per action to each region

View File

@ -120,7 +120,7 @@ public class IntegrationTestIngestWithMOB extends IntegrationTestIngest {
TableName tableName = getTablename();
try (Connection connection = ConnectionFactory.createConnection();
Admin admin = connection.getAdmin()) {
HTableDescriptor tableDesc = admin.getTableDescriptor(tableName);
HTableDescriptor tableDesc = new HTableDescriptor(admin.getDescriptor(tableName));
LOG.info("Disabling table " + getTablename());
admin.disableTable(tableName);
ColumnFamilyDescriptor mobColumn = tableDesc.getColumnFamily(mobColumnFamily);

View File

@ -278,7 +278,7 @@ public class IntegrationTestRegionReplicaPerf extends IntegrationTestBase {
// one last sanity check, then send in the clowns!
assertEquals("Table must be created with DisabledRegionSplitPolicy. Broken test.",
DisabledRegionSplitPolicy.class.getName(),
util.getAdmin().getTableDescriptor(tableName).getRegionSplitPolicyClassName());
util.getAdmin().getDescriptor(tableName).getRegionSplitPolicyClassName());
startMonkey();
// collect a baseline without region replicas.

View File

@ -261,7 +261,7 @@ public class Action {
Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
boolean result = false;
try {
result = admin.balancer();
result = admin.balance();
} catch (Exception e) {
LOG.warn("Got exception while doing balance ", e);
}

View File

@ -19,13 +19,12 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionInfo;
/**
* Region that queues a compaction of a random region from the table.
@ -55,14 +54,14 @@ public class CompactRandomRegionOfTableAction extends Action {
LOG.info("Performing action: Compact random region of table "
+ tableName + ", major=" + major);
List<HRegionInfo> regions = admin.getTableRegions(tableName);
List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.isEmpty()) {
LOG.info("Table " + tableName + " doesn't have regions to compact");
return;
}
HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
regions.toArray(new HRegionInfo[regions.size()]));
RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
regions.toArray(new RegionInfo[regions.size()]));
try {
if (major) {

View File

@ -19,12 +19,11 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionInfo;
/**
* Action that tries to flush a random region of a table.
@ -48,14 +47,14 @@ public class FlushRandomRegionOfTableAction extends Action {
Admin admin = util.getAdmin();
LOG.info("Performing action: Flush random region of table " + tableName);
List<HRegionInfo> regions = admin.getTableRegions(tableName);
List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.isEmpty()) {
LOG.info("Table " + tableName + " doesn't have regions to flush");
return;
}
HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
regions.toArray(new HRegionInfo[regions.size()]));
RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
regions.toArray(new RegionInfo[regions.size()]));
LOG.debug("Flushing region " + region.getRegionNameAsString());
try {
admin.flushRegion(region.getRegionName());

View File

@ -19,12 +19,11 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionInfo;
/**
* Action to merge regions of a table.
@ -48,15 +47,15 @@ public class MergeRandomAdjacentRegionsOfTableAction extends Action {
Admin admin = util.getAdmin();
LOG.info("Performing action: Merge random adjacent regions of table " + tableName);
List<HRegionInfo> regions = admin.getTableRegions(tableName);
List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.size() < 2) {
LOG.info("Table " + tableName + " doesn't have enough regions to merge");
return;
}
int i = RandomUtils.nextInt(0, regions.size() - 1);
HRegionInfo a = regions.get(i++);
HRegionInfo b = regions.get(i);
RegionInfo a = regions.get(i++);
RegionInfo b = regions.get(i);
LOG.debug("Merging " + a.getRegionNameAsString() + " and " + b.getRegionNameAsString());
// Don't try the merge if we're stopping

View File

@ -25,7 +25,6 @@ import java.util.EnumSet;
import java.util.List;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
@ -61,7 +60,7 @@ public class MoveRegionsOfTableAction extends Action {
ServerName[] servers = getServers(admin);
LOG.info("Performing action: Move regions of table {}", tableName);
List<HRegionInfo> regions = admin.getTableRegions(tableName);
List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.isEmpty()) {
LOG.info("Table {} doesn't have regions to move", tableName);
return;
@ -70,8 +69,7 @@ public class MoveRegionsOfTableAction extends Action {
Collections.shuffle(regions);
long start = System.currentTimeMillis();
for (HRegionInfo regionInfo:regions) {
for (RegionInfo regionInfo : regions) {
// Don't try the move if we're stopping
if (context.isStopping()) {
return;

View File

@ -19,12 +19,11 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionInfo;
/**
* Action that tries to split a random region of a table.
@ -48,7 +47,7 @@ public class SplitRandomRegionOfTableAction extends Action {
Admin admin = util.getAdmin();
LOG.info("Performing action: Split random region of table " + tableName);
List<HRegionInfo> regions = admin.getTableRegions(tableName);
List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.isEmpty()) {
LOG.info("Table " + tableName + " doesn't have regions to split");
return;
@ -58,11 +57,11 @@ public class SplitRandomRegionOfTableAction extends Action {
return;
}
HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
regions.toArray(new HRegionInfo[regions.size()]));
RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
regions.toArray(new RegionInfo[regions.size()]));
LOG.debug("Splitting region " + region.getRegionNameAsString());
try {
admin.splitRegion(region.getRegionName());
admin.splitRegionAsync(region.getRegionName(), null).get();
} catch (Exception ex) {
LOG.warn("Split failed, might be caused by other chaos: " + ex.getMessage());
}

View File

@ -670,9 +670,9 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
LOG.error("Failure in chain verification: " + msg);
try (Connection connection = ConnectionFactory.createConnection(context.getConfiguration());
Admin admin = connection.getAdmin()) {
LOG.error("cluster status:\n" + admin.getClusterStatus());
LOG.error("cluster metrics:\n" + admin.getClusterMetrics());
LOG.error("table regions:\n"
+ Joiner.on("\n").join(admin.getTableRegions(table)));
+ Joiner.on("\n").join(admin.getRegions(table)));
}
}
}

View File

@ -30,7 +30,7 @@ import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.NamespaceExistException;
@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
import org.apache.hadoop.hbase.ipc.FatalConnectionException;
import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Bytes;
@ -74,6 +73,8 @@ import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
/**
* Integration test that should benchmark how fast HBase can recover from failures. This test starts
* different threads:
@ -572,7 +573,7 @@ public class IntegrationTestMTTR {
Admin admin = null;
try {
admin = util.getAdmin();
ClusterStatus status = admin.getClusterStatus();
ClusterMetrics status = admin.getClusterMetrics();
return status != null;
} finally {
if (admin != null) {

View File

@ -70,7 +70,7 @@ public class IntegrationTestRSGroup extends TestRSGroupsBase {
deleteTableIfNecessary();
deleteNamespaceIfNecessary();
deleteGroups();
admin.setBalancerRunning(true, true);
admin.balancerSwitch(true, true);
LOG.info("Restoring the cluster");
((IntegrationTestingUtility)TEST_UTIL).restoreCluster();

View File

@ -340,7 +340,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
"Must specify an existing table for read commands. Run a write command first.");
}
HTableDescriptor desc =
exists ? admin.getTableDescriptor(TableName.valueOf(opts.tableName)) : null;
exists ? new HTableDescriptor(admin.getDescriptor(TableName.valueOf(opts.tableName))) : null;
byte[][] splits = getSplits(opts);
// recreate the table when user has requested presplit or when existing

View File

@ -700,8 +700,8 @@ public class TestHFileOutputFormat2 {
// Perform the actual load
for (HFileOutputFormat2.TableInfo singleTableInfo : tableInfo) {
Path tableDir = testDir;
String tableNameStr = singleTableInfo.getHTableDescriptor().getNameAsString();
LOG.info("Running LoadIncrementalHFiles on table" + tableNameStr);
String tableNameStr = singleTableInfo.getTableDescriptor().getTableName().getNameAsString();
LOG.info("Running BulkLoadHFiles on table" + tableNameStr);
if (writeMultipleTables) {
tableDir = new Path(testDir, tableNameStr);
}
@ -1237,7 +1237,7 @@ public class TestHFileOutputFormat2 {
// deep inspection: get the StoreFile dir
final Path storePath = new Path(
FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
new Path(admin.getRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
Bytes.toString(FAMILIES[0])));
assertEquals(0, fs.listStatus(storePath).length);
@ -1318,7 +1318,7 @@ public class TestHFileOutputFormat2 {
// deep inspection: get the StoreFile dir
final Path storePath = new Path(
FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
new Path(admin.getRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
Bytes.toString(FAMILIES[0])));
assertEquals(0, fs.listStatus(storePath).length);
@ -1411,8 +1411,10 @@ public class TestHFileOutputFormat2 {
Admin admin = c.getAdmin();
RegionLocator regionLocator = c.getRegionLocator(tname)) {
Path outDir = new Path("incremental-out");
runIncrementalPELoad(conf, Arrays.asList(new HFileOutputFormat2.TableInfo(admin
.getTableDescriptor(tname), regionLocator)), outDir, false);
runIncrementalPELoad(conf,
Arrays
.asList(new HFileOutputFormat2.TableInfo(admin.getDescriptor(tname), regionLocator)),
outDir, false);
}
} else {
throw new RuntimeException(

View File

@ -79,8 +79,8 @@ public class TestExportSnapshot {
public final TestName testName = new TestName();
protected TableName tableName;
private byte[] emptySnapshotName;
private byte[] snapshotName;
private String emptySnapshotName;
private String snapshotName;
private int tableNumFiles;
private Admin admin;
@ -117,8 +117,8 @@ public class TestExportSnapshot {
this.admin = TEST_UTIL.getAdmin();
tableName = TableName.valueOf("testtb-" + testName.getMethodName());
snapshotName = Bytes.toBytes("snaptb0-" + testName.getMethodName());
emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + testName.getMethodName());
snapshotName = "snaptb0-" + testName.getMethodName();
emptySnapshotName = "emptySnaptb0-" + testName.getMethodName();
// create Table
createTable();
@ -128,7 +128,7 @@ public class TestExportSnapshot {
// Add some rows
SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 50, FAMILY);
tableNumFiles = admin.getTableRegions(tableName).size();
tableNumFiles = admin.getRegions(tableName).size();
// take a snapshot
admin.snapshot(snapshotName, tableName);
@ -186,18 +186,18 @@ public class TestExportSnapshot {
@Test
public void testExportWithTargetName() throws Exception {
final byte[] targetName = Bytes.toBytes("testExportWithTargetName");
final String targetName = "testExportWithTargetName";
testExportFileSystemState(tableName, snapshotName, targetName, tableNumFiles);
}
private void testExportFileSystemState(final TableName tableName, final byte[] snapshotName,
final byte[] targetName, int filesExpected) throws Exception {
private void testExportFileSystemState(final TableName tableName, final String snapshotName,
final String targetName, int filesExpected) throws Exception {
testExportFileSystemState(tableName, snapshotName, targetName,
filesExpected, getHdfsDestinationDir(), false);
}
protected void testExportFileSystemState(final TableName tableName,
final byte[] snapshotName, final byte[] targetName, int filesExpected,
final String snapshotName, final String targetName, int filesExpected,
Path copyDir, boolean overwrite) throws Exception {
testExportFileSystemState(TEST_UTIL.getConfiguration(), tableName, snapshotName, targetName,
filesExpected, TEST_UTIL.getDefaultRootDirPath(), copyDir,
@ -208,7 +208,7 @@ public class TestExportSnapshot {
* Creates destination directory, runs ExportSnapshot() tool, and runs some verifications.
*/
protected static void testExportFileSystemState(final Configuration conf, final TableName tableName,
final byte[] snapshotName, final byte[] targetName, final int filesExpected,
final String snapshotName, final String targetName, final int filesExpected,
final Path sourceDir, Path copyDir, final boolean overwrite,
final RegionPredicate bypassregionPredicate, boolean success) throws Exception {
URI hdfsUri = FileSystem.get(conf).getUri();
@ -217,12 +217,12 @@ public class TestExportSnapshot {
List<String> opts = new ArrayList<>();
opts.add("--snapshot");
opts.add(Bytes.toString(snapshotName));
opts.add(snapshotName);
opts.add("--copy-to");
opts.add(copyDir.toString());
if (targetName != snapshotName) {
if (!targetName.equals(snapshotName)) {
opts.add("--target");
opts.add(Bytes.toString(targetName));
opts.add(targetName);
}
if (overwrite) opts.add("--overwrite");
@ -230,7 +230,7 @@ public class TestExportSnapshot {
int res = run(conf, new ExportSnapshot(), opts.toArray(new String[opts.size()]));
assertEquals(success ? 0 : 1, res);
if (!success) {
final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName));
final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, targetName);
assertFalse(fs.exists(new Path(copyDir, targetDir)));
return;
}
@ -247,12 +247,12 @@ public class TestExportSnapshot {
// compare the snapshot metadata and verify the hfiles
final FileSystem hdfs = FileSystem.get(hdfsUri, conf);
final Path snapshotDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(snapshotName));
final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName));
final Path snapshotDir = new Path(HConstants.SNAPSHOT_DIR_NAME, snapshotName);
final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, targetName);
verifySnapshotDir(hdfs, new Path(sourceDir, snapshotDir),
fs, new Path(copyDir, targetDir));
Set<String> snapshotFiles = verifySnapshot(conf, fs, copyDir, tableName,
Bytes.toString(targetName), bypassregionPredicate);
targetName, bypassregionPredicate);
assertEquals(filesExpected, snapshotFiles.size());
}

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock;
import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
@ -98,7 +97,7 @@ public class TestExportSnapshotNoCluster {
builder.commit();
int snapshotFilesCount = r1Files.length + r2Files.length;
byte[] snapshotName = Bytes.toBytes(builder.getSnapshotDescription().getName());
String snapshotName = builder.getSnapshotDescription().getName();
TableName tableName = builder.getTableDescriptor().getTableName();
TestExportSnapshot.testExportFileSystemState(TEST_UTIL.getConfiguration(),
tableName, snapshotName, snapshotName, snapshotFilesCount,

View File

@ -19,7 +19,7 @@
package org.apache.hadoop.hbase.rest;
import java.io.IOException;
import java.util.List;
import javax.servlet.ServletContext;
import javax.ws.rs.Consumes;
import javax.ws.rs.DELETE;
@ -33,16 +33,16 @@ import javax.ws.rs.core.Context;
import javax.ws.rs.core.HttpHeaders;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.UriInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.rest.model.NamespacesInstanceModel;
import org.apache.hadoop.hbase.rest.model.TableListModel;
import org.apache.hadoop.hbase.rest.model.TableModel;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Implements the following REST end points:
@ -100,9 +100,10 @@ public class NamespacesInstanceResource extends ResourceBase {
if(queryTables){
TableListModel tableModel = new TableListModel();
try{
HTableDescriptor[] tables = servlet.getAdmin().listTableDescriptorsByNamespace(namespace);
for(int i = 0; i < tables.length; i++){
tableModel.add(new TableModel(tables[i].getTableName().getQualifierAsString()));
List<TableDescriptor> tables =
servlet.getAdmin().listTableDescriptorsByNamespace(Bytes.toBytes(namespace));
for (TableDescriptor table : tables) {
tableModel.add(new TableModel(table.getTableName().getQualifierAsString()));
}
servlet.getMetrics().incrementSucessfulGetRequests(1);

View File

@ -124,7 +124,7 @@ public class SchemaResource extends ResourceBase {
}
if (admin.tableExists(name)) {
admin.disableTable(name);
admin.modifyTable(name, htd);
admin.modifyTable(htd);
admin.enableTable(name);
servlet.getMetrics().incrementSucessfulPutRequests(1);
} else try {
@ -152,7 +152,7 @@ public class SchemaResource extends ResourceBase {
.build();
}
try {
HTableDescriptor htd = admin.getTableDescriptor(name);
HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(name));
admin.disableTable(name);
try {
for (ColumnSchemaModel family: model.getColumns()) {

View File

@ -524,7 +524,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
* @throws IOException
*/
private boolean checkTable(RemoteAdmin admin) throws IOException {
HTableDescriptor tableDescriptor = getTableDescriptor();
HTableDescriptor tableDescriptor = getDescriptor();
if (this.presplitRegions > 0) {
// presplit requested
if (admin.isTableAvailable(tableDescriptor.getTableName().getName())) {
@ -548,7 +548,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
return tableExists;
}
protected HTableDescriptor getTableDescriptor() {
protected HTableDescriptor getDescriptor() {
if (TABLE_DESCRIPTOR == null) {
TABLE_DESCRIPTOR = new HTableDescriptor(tableName);
HColumnDescriptor family = new HColumnDescriptor(FAMILY_NAME);

View File

@ -51,7 +51,7 @@ public interface BulkLoadObserver {
* It can't bypass the default action, e.g., ctx.bypass() won't have effect.
* If you need to get the region or table name, get it from the
* <code>ctx</code> as follows: <code>code>ctx.getEnvironment().getRegion()</code>. Use
* getRegionInfo to fetch the encodedName and use getTableDescriptor() to get the tableName.
* getRegionInfo to fetch the encodedName and use getDescriptor() to get the tableName.
* @param ctx the environment to interact with the framework and master
*/
default void prePrepareBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx)
@ -62,7 +62,7 @@ public interface BulkLoadObserver {
* It can't bypass the default action, e.g., ctx.bypass() won't have effect.
* If you need to get the region or table name, get it from the
* <code>ctx</code> as follows: <code>code>ctx.getEnvironment().getRegion()</code>. Use
* getRegionInfo to fetch the encodedName and use getTableDescriptor() to get the tableName.
* getRegionInfo to fetch the encodedName and use getDescriptor() to get the tableName.
* @param ctx the environment to interact with the framework and master
*/
default void preCleanupBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx)

View File

@ -202,7 +202,7 @@ public class CatalogJanitor extends ScheduledChore {
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
Path rootdir = this.services.getMasterFileSystem().getRootDir();
Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
TableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
TableDescriptor htd = getDescriptor(mergedRegion.getTable());
HRegionFileSystem regionFs = null;
try {
regionFs = HRegionFileSystem.openRegionFromFileSystem(
@ -410,7 +410,7 @@ public class CatalogJanitor extends ScheduledChore {
}
boolean references = false;
TableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
TableDescriptor parentDescriptor = getDescriptor(parent.getTable());
try {
regionFs = HRegionFileSystem.openRegionFromFileSystem(
this.services.getConfiguration(), fs, tabledir, daughter, true);
@ -428,7 +428,7 @@ public class CatalogJanitor extends ScheduledChore {
return new Pair<>(Boolean.TRUE, Boolean.valueOf(references));
}
private TableDescriptor getTableDescriptor(final TableName tableName)
private TableDescriptor getDescriptor(final TableName tableName)
throws FileNotFoundException, IOException {
return this.services.getTableDescriptors().get(tableName);
}

View File

@ -229,7 +229,7 @@ public class RegionStateStore {
// ============================================================================================
public void splitRegion(RegionInfo parent, RegionInfo hriA, RegionInfo hriB,
ServerName serverName) throws IOException {
TableDescriptor htd = getTableDescriptor(parent.getTable());
TableDescriptor htd = getDescriptor(parent.getTable());
long parentOpenSeqNum = HConstants.NO_SEQNUM;
if (htd.hasGlobalReplicationScope()) {
parentOpenSeqNum = getOpenSeqNumForParentRegion(parent);
@ -243,7 +243,7 @@ public class RegionStateStore {
// ============================================================================================
public void mergeRegions(RegionInfo child, RegionInfo hriA, RegionInfo hriB,
ServerName serverName) throws IOException {
TableDescriptor htd = getTableDescriptor(child.getTable());
TableDescriptor htd = getDescriptor(child.getTable());
long regionAOpenSeqNum = -1L;
long regionBOpenSeqNum = -1L;
if (htd.hasGlobalReplicationScope()) {
@ -269,7 +269,7 @@ public class RegionStateStore {
// Table Descriptors helpers
// ==========================================================================
private boolean hasGlobalReplicationScope(TableName tableName) throws IOException {
return hasGlobalReplicationScope(getTableDescriptor(tableName));
return hasGlobalReplicationScope(getDescriptor(tableName));
}
private boolean hasGlobalReplicationScope(TableDescriptor htd) {
@ -280,7 +280,7 @@ public class RegionStateStore {
return htd != null ? htd.getRegionReplication() : 1;
}
private TableDescriptor getTableDescriptor(TableName tableName) throws IOException {
private TableDescriptor getDescriptor(TableName tableName) throws IOException {
return master.getTableDescriptors().get(tableName);
}

View File

@ -192,7 +192,7 @@ class RegionLocationFinder {
*/
protected HDFSBlocksDistribution internalGetTopBlockLocation(RegionInfo region) {
try {
TableDescriptor tableDescriptor = getTableDescriptor(region.getTable());
TableDescriptor tableDescriptor = getDescriptor(region.getTable());
if (tableDescriptor != null) {
HDFSBlocksDistribution blocksDistribution =
HRegion.computeHDFSBlocksDistribution(getConf(), tableDescriptor, region);
@ -213,7 +213,7 @@ class RegionLocationFinder {
* @return TableDescriptor
* @throws IOException
*/
protected TableDescriptor getTableDescriptor(TableName tableName) throws IOException {
protected TableDescriptor getDescriptor(TableName tableName) throws IOException {
TableDescriptor tableDescriptor = null;
try {
if (this.services != null && this.services.getTableDescriptors() != null) {

View File

@ -18,9 +18,7 @@
*/
package org.apache.hadoop.hbase.master.normalizer;
import java.io.IOException;
import java.util.Arrays;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
@ -78,8 +76,8 @@ public class SplitNormalizationPlan implements NormalizationPlan {
public void execute(Admin admin) {
LOG.info("Executing splitting normalization plan: " + this);
try {
admin.splitRegion(regionInfo.getRegionName());
} catch (IOException ex) {
admin.splitRegionAsync(regionInfo.getRegionName(), null).get();
} catch (Exception ex) {
LOG.error("Error during region split: ", ex);
}
}

View File

@ -111,11 +111,7 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
cleanExpiredMobFiles(tableName, family);
return 0;
} finally {
try {
admin.close();
} catch (IOException e) {
LOG.error("Failed to close the HBaseAdmin.", e);
}
admin.close();
try {
connection.close();
} catch (IOException e) {

View File

@ -663,8 +663,8 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
@Override
public void run() {
Preconditions.checkNotNull(server);
if (server.isStopped()
|| (region.getTableDescriptor() != null && !region.getTableDescriptor().isCompactionEnabled())) {
if (server.isStopped() || (region.getTableDescriptor() != null &&
!region.getTableDescriptor().isCompactionEnabled())) {
region.decrementCompactionsQueuedCount();
return;
}

View File

@ -53,8 +53,8 @@ public class DelimitedKeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundR
// read the prefix length from the table descriptor
String delimiterString = region.getTableDescriptor().getValue(DELIMITER_KEY);
if (delimiterString == null || delimiterString.length() == 0) {
LOG.error(DELIMITER_KEY + " not specified for table " + region.getTableDescriptor().getTableName() +
". Using default RegionSplitPolicy");
LOG.error(DELIMITER_KEY + " not specified for table " +
region.getTableDescriptor().getTableName() + ". Using default RegionSplitPolicy");
return;
}
delimiter = Bytes.toBytes(delimiterString);

View File

@ -130,7 +130,7 @@ public final class SnapshotManifest {
/**
* Return a SnapshotManifest instance with the information already loaded in-memory.
* SnapshotManifest manifest = SnapshotManifest.open(...)
* TableDescriptor htd = manifest.getTableDescriptor()
* TableDescriptor htd = manifest.getDescriptor()
* for (SnapshotRegionManifest regionManifest: manifest.getRegionManifests())
* hri = regionManifest.getRegionInfo()
* for (regionManifest.getFamilyFiles())

View File

@ -90,7 +90,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooKeeper;
@ -99,6 +98,7 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
/**
@ -1202,13 +1202,13 @@ public final class Canary implements Tool {
if (this.useRegExp) {
Pattern pattern = null;
TableDescriptor[] tds = null;
List<TableDescriptor> tds = null;
Set<String> tmpTables = new TreeSet<>();
try {
LOG.debug(String.format("reading list of tables"));
tds = this.admin.listTables(pattern);
tds = this.admin.listTableDescriptors(pattern);
if (tds == null) {
tds = new TableDescriptor[0];
tds = Collections.emptyList();
}
for (String monitorTarget : monitorTargets) {
pattern = Pattern.compile(monitorTarget);
@ -1293,7 +1293,7 @@ public final class Canary implements Tool {
}
int numberOfCoveredServers = serverSet.size();
if (numberOfCoveredServers < numberOfServers) {
admin.balancer();
admin.balance();
}
}

View File

@ -2291,7 +2291,7 @@ public class HBaseFsck extends Configured implements Closeable {
if (hi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
return;
}
int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
int numReplicas = admin.getDescriptor(hi.getTableName()).getRegionReplication();
for (int i = 1; i < numReplicas; i++) {
if (hi.getPrimaryHRIForDeployedReplica() == null) continue;
RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
@ -2344,7 +2344,7 @@ public class HBaseFsck extends Configured implements Closeable {
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER);
// also get the locations of the replicas to close if the primary region is being closed
if (hi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
int numReplicas = admin.getDescriptor(hi.getTableName()).getRegionReplication();
for (int i = 0; i < numReplicas; i++) {
get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i));
get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(i));
@ -2395,7 +2395,7 @@ public class HBaseFsck extends Configured implements Closeable {
// also assign replicas if needed (do it only when this call operates on a primary replica)
if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) return;
int replicationCount = admin.getTableDescriptor(hri.getTable()).getRegionReplication();
int replicationCount = admin.getDescriptor(hri.getTable()).getRegionReplication();
for (int i = 1; i < replicationCount; i++) {
hri = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
HbckInfo h = regionInfoMap.get(hri.getEncodedName());
@ -2512,7 +2512,7 @@ public class HBaseFsck extends Configured implements Closeable {
}
}
LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI());
int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication();
HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet(), numReplicas);
@ -2540,7 +2540,7 @@ public class HBaseFsck extends Configured implements Closeable {
}
LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI());
int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication();
HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet(), numReplicas);
@ -3660,7 +3660,7 @@ public class HBaseFsck extends Configured implements Closeable {
metaRegions.put(value.getReplicaId(), value);
}
}
int metaReplication = admin.getTableDescriptor(TableName.META_TABLE_NAME)
int metaReplication = admin.getDescriptor(TableName.META_TABLE_NAME)
.getRegionReplication();
boolean noProblem = true;
// There will be always entries in regionInfoMap corresponding to hbase:meta & its replicas

View File

@ -400,12 +400,12 @@
</div>
<% if (rsGroupTables != null && rsGroupTables.size() > 0) {
HTableDescriptor[] tables = null;
List<TableDescriptor> tables;
try (Admin admin = master.getConnection().getAdmin()) {
tables = master.isInitialized() ? admin.listTables((Pattern)null, true) : null;
tables = master.isInitialized() ? admin.listTableDescriptors((Pattern)null, true) : null;
}
Map<TableName, HTableDescriptor> tableDescriptors
= Stream.of(tables).collect(Collectors.toMap(TableDescriptor::getTableName, p -> p));
= tables.stream().collect(Collectors.toMap(TableDescriptor::getTableName, p -> new HTableDescriptor(p)));
%>
<table class="table table-striped">
<tr>

View File

@ -172,7 +172,7 @@ if ( fqtn != null ) {
%> Compact request accepted. <%
} else if (action.equals("merge")) {
if (left != null && left.length() > 0 && right != null && right.length() > 0) {
admin.mergeRegions(Bytes.toBytesBinary(left), Bytes.toBytesBinary(right), false);
admin.mergeRegionsAsync(Bytes.toBytesBinary(left), Bytes.toBytesBinary(right), false);
}
%> Merge request accepted. <%
}

View File

@ -1821,31 +1821,14 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
}
/**
* Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
* Modify a table, synchronous.
* @deprecated just use {@link Admin#modifyTable(TableDescriptor)} directly as it is synchronous
* now.
*/
@SuppressWarnings("serial")
@Deprecated
public static void modifyTableSync(Admin admin, TableDescriptor desc)
throws IOException, InterruptedException {
admin.modifyTable(desc);
Pair<Integer, Integer> status = new Pair<Integer, Integer>() {{
setFirst(0);
setSecond(0);
}};
int i = 0;
do {
status = admin.getAlterStatus(desc.getTableName());
if (status.getSecond() != 0) {
LOG.debug(status.getSecond() - status.getFirst() + "/" + status.getSecond()
+ " regions updated.");
Thread.sleep(1 * 1000L);
} else {
LOG.debug("All regions updated.");
break;
}
} while (status.getFirst() != 0 && i++ < 500);
if (status.getFirst() != 0) {
throw new IOException("Failed to update all regions even after 500 seconds.");
}
}
/**
@ -1854,9 +1837,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
public static void setReplicas(Admin admin, TableName table, int replicaCount)
throws IOException, InterruptedException {
admin.disableTable(table);
HTableDescriptor desc = new HTableDescriptor(admin.getTableDescriptor(table));
HTableDescriptor desc = new HTableDescriptor(admin.getDescriptor(table));
desc.setRegionReplication(replicaCount);
admin.modifyTable(desc.getTableName(), desc);
admin.modifyTable(desc);
admin.enableTable(table);
}

View File

@ -22,7 +22,6 @@ import java.util.EnumSet;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.Waiter.Predicate;
@ -82,25 +81,6 @@ public class TestClientClusterStatus {
}
}
@Test
public void testDefaults() throws Exception {
ClusterStatus origin = ADMIN.getClusterStatus();
ClusterStatus defaults
= new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.allOf(Option.class)));
checkPbObjectNotNull(origin);
checkPbObjectNotNull(defaults);
Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
Assert.assertTrue(origin.getMasterInfoPort() == defaults.getMasterInfoPort());
Assert.assertTrue(origin.equals(defaults));
Assert.assertTrue(origin.getServersName().size() == defaults.getServersName().size());
}
@Test
public void testNone() throws Exception {
ClusterMetrics status0 = ADMIN.getClusterMetrics(EnumSet.allOf(Option.class));
@ -206,8 +186,8 @@ public class TestClientClusterStatus {
public void testObserver() throws IOException {
int preCount = MyObserver.PRE_COUNT.get();
int postCount = MyObserver.POST_COUNT.get();
Assert.assertTrue(Stream.of(ADMIN.getClusterStatus().getMasterCoprocessors())
.anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
Assert.assertTrue(ADMIN.getClusterMetrics().getMasterCoprocessorNames().stream()
.anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
Assert.assertEquals(preCount + 1, MyObserver.PRE_COUNT.get());
Assert.assertEquals(postCount + 1, MyObserver.POST_COUNT.get());
}

View File

@ -23,14 +23,17 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.regex.Pattern;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
@ -84,7 +87,7 @@ public class TestNamespace {
@Before
public void beforeMethod() throws IOException {
for (HTableDescriptor desc : admin.listTables(prefix+".*")) {
for (TableDescriptor desc : admin.listTableDescriptors(Pattern.compile(prefix + ".*"))) {
admin.disableTable(desc.getTableName());
admin.deleteTable(desc.getTableName());
}
@ -112,14 +115,14 @@ public class TestNamespace {
//verify existence of system tables
Set<TableName> systemTables = Sets.newHashSet(
TableName.META_TABLE_NAME);
HTableDescriptor[] descs =
admin.listTableDescriptorsByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE.getName());
assertEquals(systemTables.size(), descs.length);
for (HTableDescriptor desc : descs) {
List<TableDescriptor> descs = admin.listTableDescriptorsByNamespace(
Bytes.toBytes(NamespaceDescriptor.SYSTEM_NAMESPACE.getName()));
assertEquals(systemTables.size(), descs.size());
for (TableDescriptor desc : descs) {
assertTrue(systemTables.contains(desc.getTableName()));
}
//verify system tables aren't listed
assertEquals(0, admin.listTables().length);
assertEquals(0, admin.listTableDescriptors().size());
//Try creating default and system namespaces.
boolean exceptionCaught = false;
@ -189,15 +192,15 @@ public class TestNamespace {
admin.createNamespace(NamespaceDescriptor.create(nsName).build());
TEST_UTIL.createTable(tableName, Bytes.toBytes(nsName));
TEST_UTIL.createTable(tableNameFoo,Bytes.toBytes(nsName));
assertEquals(2, admin.listTables().length);
assertEquals(2, admin.listTableDescriptors().size());
assertNotNull(admin
.getTableDescriptor(tableName));
.getDescriptor(tableName));
assertNotNull(admin
.getTableDescriptor(tableNameFoo));
.getDescriptor(tableNameFoo));
//remove namespace and verify
admin.disableTable(tableName);
admin.deleteTable(tableName);
assertEquals(1, admin.listTables().length);
assertEquals(1, admin.listTableDescriptors().size());
}
@Test
@ -222,7 +225,7 @@ public class TestNamespace {
new Path(master.getMasterFileSystem().getRootDir(),
new Path(HConstants.BASE_NAMESPACE_DIR,
new Path(nsName, desc.getTableName().getQualifierAsString())))));
assertEquals(1, admin.listTables().length);
assertEquals(1, admin.listTableDescriptors().size());
//verify non-empty namespace can't be removed
try {
@ -253,7 +256,7 @@ public class TestNamespace {
HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
desc.addFamily(colDesc);
admin.createTable(desc);
assertTrue(admin.listTables().length == 1);
assertTrue(admin.listTableDescriptors().size() == 1);
admin.disableTable(desc.getTableName());
admin.deleteTable(desc.getTableName());
}
@ -265,7 +268,7 @@ public class TestNamespace {
HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
desc.addFamily(colDesc);
admin.createTable(desc);
assertEquals(0, admin.listTables().length);
assertEquals(0, admin.listTableDescriptors().size());
assertTrue(admin.tableExists(tableName));
admin.disableTable(desc.getTableName());
admin.deleteTable(desc.getTableName());
@ -338,15 +341,16 @@ public class TestNamespace {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(prefix + "ns1", name.getMethodName()));
htd.addFamily(new HColumnDescriptor("family1"));
admin.createTable(htd);
HTableDescriptor[] htds = admin.listTableDescriptorsByNamespace(prefix + "ns1");
List<TableDescriptor> htds =
admin.listTableDescriptorsByNamespace(Bytes.toBytes(prefix + "ns1"));
assertNotNull("Should have not returned null", htds);
assertEquals("Should have returned non-empty array", 1, htds.length);
assertEquals("Should have returned non-empty array", 1, htds.size());
// get table descriptors for non-existing namespace
runWithExpectedException(new Callable<Void>() {
@Override
public Void call() throws Exception {
admin.listTableDescriptorsByNamespace("non_existant_namespace");
admin.listTableDescriptorsByNamespace(Bytes.toBytes("non_existant_namespace"));
return null;
}
}, NamespaceNotFoundException.class);

View File

@ -118,7 +118,7 @@ public class TestPartialResultsFromClientSide {
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.getConfiguration().setLong(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, timeout);
TEST_UTIL.startMiniCluster(MINICLUSTER_SIZE);
TEST_UTIL.getAdmin().setBalancerRunning(false, true);
TEST_UTIL.getAdmin().balancerSwitch(false, true);
TABLE = createTestTable(TABLE_NAME, ROWS, FAMILIES, QUALIFIERS, VALUE);
}

View File

@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
@ -70,7 +71,7 @@ public class TestRegionLoad {
UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", MSG_INTERVAL);
UTIL.startMiniCluster(4);
admin = UTIL.getAdmin();
admin.setBalancerRunning(false, true);
admin.balancerSwitch(false, true);
createTables();
}
@ -95,7 +96,7 @@ public class TestRegionLoad {
// Check if regions match with the regionLoad from the server
for (ServerName serverName : admin
.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()) {
List<HRegionInfo> regions = admin.getOnlineRegions(serverName);
List<RegionInfo> regions = admin.getRegions(serverName);
LOG.info("serverName=" + serverName + ", regions=" +
regions.stream().map(r -> r.getRegionNameAsString()).collect(Collectors.toList()));
Collection<RegionLoad> regionLoads = admin.getRegionMetrics(serverName)
@ -108,7 +109,7 @@ public class TestRegionLoad {
// Check if regionLoad matches the table's regions and nothing is missed
for (TableName table : new TableName[]{TABLE_1, TABLE_2, TABLE_3}) {
List<HRegionInfo> tableRegions = admin.getTableRegions(table);
List<RegionInfo> tableRegions = admin.getRegions(table);
List<RegionLoad> regionLoads = Lists.newArrayList();
for (ServerName serverName : admin
@ -159,23 +160,21 @@ public class TestRegionLoad {
assertEquals("regionLoads from SN should be empty", 0, regionLoads.size());
}
private void checkRegionsAndRegionLoads(Collection<HRegionInfo> regions,
private void checkRegionsAndRegionLoads(Collection<RegionInfo> regions,
Collection<RegionLoad> regionLoads) {
for (RegionLoad load : regionLoads) {
assertNotNull(load.regionLoadPB);
}
assertEquals("No of regions and regionloads doesn't match",
regions.size(), regionLoads.size());
assertEquals("No of regions and regionloads doesn't match", regions.size(), regionLoads.size());
Map<byte[], RegionLoad> regionLoadMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
for (RegionLoad regionLoad : regionLoads) {
regionLoadMap.put(regionLoad.getName(), regionLoad);
}
for (HRegionInfo info : regions) {
for (RegionInfo info : regions) {
assertTrue("Region not in regionLoadMap region:" + info.getRegionNameAsString() +
" regionMap: " + regionLoadMap, regionLoadMap.containsKey(info.getRegionName()));
" regionMap: " + regionLoadMap, regionLoadMap.containsKey(info.getRegionName()));
}
}
}

View File

@ -34,7 +34,7 @@ public class CloneSnapshotFromClientNormalTestBase extends CloneSnapshotFromClie
testCloneSnapshot(clonedTableName, emptySnapshot, 0);
}
private void testCloneSnapshot(TableName tableName, byte[] snapshotName, int snapshotRows)
private void testCloneSnapshot(TableName tableName, String snapshotName, int snapshotRows)
throws IOException, InterruptedException {
// create a new table from snapshot
admin.cloneSnapshot(snapshotName, tableName);

View File

@ -40,10 +40,10 @@ public class CloneSnapshotFromClientTestBase {
protected final byte[] FAMILY = Bytes.toBytes("cf");
protected byte[] emptySnapshot;
protected byte[] snapshotName0;
protected byte[] snapshotName1;
protected byte[] snapshotName2;
protected String emptySnapshot;
protected String snapshotName0;
protected String snapshotName1;
protected String snapshotName2;
protected TableName tableName;
protected int snapshot0Rows;
protected int snapshot1Rows;
@ -87,10 +87,10 @@ public class CloneSnapshotFromClientTestBase {
long tid = System.currentTimeMillis();
tableName = TableName.valueOf(getValidMethodName() + tid);
emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
emptySnapshot = "emptySnaptb-" + tid;
snapshotName0 = "snaptb0-" + tid;
snapshotName1 = "snaptb1-" + tid;
snapshotName2 = "snaptb2-" + tid;
createTableAndSnapshots();
}

View File

@ -24,7 +24,6 @@ import java.io.IOException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
public class RestoreSnapshotFromClientSimpleTestBase extends RestoreSnapshotFromClientTestBase {
@ -63,7 +62,7 @@ public class RestoreSnapshotFromClientSimpleTestBase extends RestoreSnapshotFrom
@Test
public void testCorruptedSnapshot() throws IOException, InterruptedException {
SnapshotTestingUtils.corruptSnapshot(TEST_UTIL, Bytes.toString(snapshotName0));
SnapshotTestingUtils.corruptSnapshot(TEST_UTIL, snapshotName0);
TableName cloneName =
TableName.valueOf(getValidMethodName() + "-" + System.currentTimeMillis());
try {

View File

@ -42,10 +42,10 @@ public class RestoreSnapshotFromClientTestBase {
protected final byte[] TEST_FAMILY2 = Bytes.toBytes("cf2");
protected TableName tableName;
protected byte[] emptySnapshot;
protected byte[] snapshotName0;
protected byte[] snapshotName1;
protected byte[] snapshotName2;
protected String emptySnapshot;
protected String snapshotName0;
protected String snapshotName1;
protected String snapshotName2;
protected int snapshot0Rows;
protected int snapshot1Rows;
protected Admin admin;
@ -84,10 +84,10 @@ public class RestoreSnapshotFromClientTestBase {
long tid = System.currentTimeMillis();
tableName = TableName.valueOf(getValidMethodName() + "-" + tid);
emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
emptySnapshot = "emptySnaptb-" + tid;
snapshotName0 = "snaptb0-" + tid;
snapshotName1 = "snaptb1-" + tid;
snapshotName2 = "snaptb2-" + tid;
// create Table and disable it
createTable();

View File

@ -31,6 +31,7 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Pattern;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
@ -59,7 +60,6 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Rule;
@ -86,7 +86,7 @@ public class TestAdmin1 {
private static final Logger LOG = LoggerFactory.getLogger(TestAdmin1.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private Admin admin;
private static Admin ADMIN;
@Rule
public TestName name = new TestName();
@ -98,21 +98,16 @@ public class TestAdmin1 {
TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
TEST_UTIL.startMiniCluster(3);
ADMIN = TEST_UTIL.getAdmin();
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Before
public void setUp() throws Exception {
this.admin = TEST_UTIL.getAdmin();
}
@After
public void tearDown() throws Exception {
for (HTableDescriptor htd : this.admin.listTables()) {
for (TableDescriptor htd : ADMIN.listTableDescriptors()) {
TEST_UTIL.deleteTable(htd.getTableName());
}
}
@ -122,7 +117,7 @@ public class TestAdmin1 {
final TableName unknowntable = TableName.valueOf(name.getMethodName());
Exception exception = null;
try {
this.admin.compact(unknowntable);
ADMIN.compact(unknowntable);
} catch (IOException e) {
exception = e;
}
@ -130,7 +125,7 @@ public class TestAdmin1 {
exception = null;
try {
this.admin.flush(unknowntable);
ADMIN.flush(unknowntable);
} catch (IOException e) {
exception = e;
}
@ -138,7 +133,7 @@ public class TestAdmin1 {
exception = null;
try {
this.admin.split(unknowntable);
ADMIN.split(unknowntable);
} catch (IOException e) {
exception = e;
}
@ -153,7 +148,7 @@ public class TestAdmin1 {
HColumnDescriptor nonexistentHcd = new HColumnDescriptor(nonexistentColumn);
Exception exception = null;
try {
this.admin.addColumnFamily(nonexistentTable, nonexistentHcd);
ADMIN.addColumnFamily(nonexistentTable, nonexistentHcd);
} catch (IOException e) {
exception = e;
}
@ -161,7 +156,7 @@ public class TestAdmin1 {
exception = null;
try {
this.admin.deleteTable(nonexistentTable);
ADMIN.deleteTable(nonexistentTable);
} catch (IOException e) {
exception = e;
}
@ -169,7 +164,7 @@ public class TestAdmin1 {
exception = null;
try {
this.admin.deleteColumnFamily(nonexistentTable, nonexistentColumn);
ADMIN.deleteColumnFamily(nonexistentTable, nonexistentColumn);
} catch (IOException e) {
exception = e;
}
@ -177,7 +172,7 @@ public class TestAdmin1 {
exception = null;
try {
this.admin.disableTable(nonexistentTable);
ADMIN.disableTable(nonexistentTable);
} catch (IOException e) {
exception = e;
}
@ -185,7 +180,7 @@ public class TestAdmin1 {
exception = null;
try {
this.admin.enableTable(nonexistentTable);
ADMIN.enableTable(nonexistentTable);
} catch (IOException e) {
exception = e;
}
@ -193,7 +188,7 @@ public class TestAdmin1 {
exception = null;
try {
this.admin.modifyColumnFamily(nonexistentTable, nonexistentHcd);
ADMIN.modifyColumnFamily(nonexistentTable, nonexistentHcd);
} catch (IOException e) {
exception = e;
}
@ -203,7 +198,7 @@ public class TestAdmin1 {
try {
HTableDescriptor htd = new HTableDescriptor(nonexistentTable);
htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
this.admin.modifyTable(htd.getTableName(), htd);
ADMIN.modifyTable(htd);
} catch (IOException e) {
exception = e;
}
@ -214,11 +209,11 @@ public class TestAdmin1 {
final TableName tableName = TableName.valueOf(name.getMethodName() + System.currentTimeMillis());
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(new HColumnDescriptor("cf"));
this.admin.createTable(htd);
ADMIN.createTable(htd);
try {
exception = null;
try {
this.admin.deleteColumnFamily(htd.getTableName(), nonexistentHcd.getName());
ADMIN.deleteColumnFamily(htd.getTableName(), nonexistentHcd.getName());
} catch (IOException e) {
exception = e;
}
@ -227,15 +222,15 @@ public class TestAdmin1 {
exception = null;
try {
this.admin.modifyColumnFamily(htd.getTableName(), nonexistentHcd);
ADMIN.modifyColumnFamily(htd.getTableName(), nonexistentHcd);
} catch (IOException e) {
exception = e;
}
assertTrue("found=" + exception.getClass().getName(),
exception instanceof InvalidFamilyOperationException);
} finally {
this.admin.disableTable(tableName);
this.admin.deleteTable(tableName);
ADMIN.disableTable(tableName);
ADMIN.deleteTable(tableName);
}
}
@ -253,7 +248,7 @@ public class TestAdmin1 {
get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
ht.get(get);
this.admin.disableTable(ht.getName());
ADMIN.disableTable(ht.getName());
assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
.getMaster().getTableStateManager().isTableState(
ht.getName(), TableState.State.DISABLED));
@ -281,7 +276,7 @@ public class TestAdmin1 {
ok = true;
}
assertTrue(ok);
this.admin.enableTable(table);
ADMIN.enableTable(table);
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
.getMaster().getTableStateManager().isTableState(
ht.getName(), TableState.State.ENABLED));
@ -322,7 +317,10 @@ public class TestAdmin1 {
ht1.get(get);
ht2.get(get);
this.admin.disableTables("testDisableAndEnableTable.*");
TableName[] tableNames = ADMIN.listTableNames(Pattern.compile("testDisableAndEnableTable.*"));
for (TableName tableName: tableNames) {
ADMIN.disableTable(tableName);
}
// Test that tables are disabled
get = new Get(row);
@ -340,7 +338,9 @@ public class TestAdmin1 {
assertTrue(ok);
this.admin.enableTables("testDisableAndEnableTable.*");
for (TableName tableName: tableNames) {
ADMIN.enableTable(tableName);
}
// Test that tables are enabled
try {
@ -364,12 +364,12 @@ public class TestAdmin1 {
@Test
public void testCreateTable() throws IOException {
HTableDescriptor [] tables = admin.listTables();
int numTables = tables.length;
List<TableDescriptor> tables = ADMIN.listTableDescriptors();
int numTables = tables.size();
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
tables = this.admin.listTables();
assertEquals(numTables + 1, tables.length);
tables = ADMIN.listTableDescriptors();
assertEquals(numTables + 1, tables.size());
assertTrue("Table must be enabled.",
TEST_UTIL.getHBaseCluster().getMaster().getTableStateManager()
.isTableState(tableName, TableState.State.ENABLED));
@ -403,8 +403,8 @@ public class TestAdmin1 {
assertEquals(3, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
// Truncate & Verify
this.admin.disableTable(tableName);
this.admin.truncateTable(tableName, preserveSplits);
ADMIN.disableTable(tableName);
ADMIN.truncateTable(tableName, preserveSplits);
table = TEST_UTIL.getConnection().getTable(tableName);
try {
assertEquals(0, TEST_UTIL.countRows(table));
@ -427,7 +427,7 @@ public class TestAdmin1 {
htd.addFamily(fam1);
htd.addFamily(fam2);
htd.addFamily(fam3);
this.admin.createTable(htd);
ADMIN.createTable(htd);
Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
TableDescriptor confirmedHtd = table.getDescriptor();
assertEquals(0, TableDescriptor.COMPARATOR.compare(htd, confirmedHtd));
@ -441,19 +441,19 @@ public class TestAdmin1 {
final TableName tableName = TableName.valueOf(name.getMethodName());
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(fam1);
this.admin.createTable(htd);
ADMIN.createTable(htd);
Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
long ts = this.admin.getLastMajorCompactionTimestamp(tableName);
long ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
assertEquals(0, ts);
Put p = new Put(Bytes.toBytes("row1"));
p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
table.put(p);
ts = this.admin.getLastMajorCompactionTimestamp(tableName);
ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
// no files written -> no data
assertEquals(0, ts);
this.admin.flush(tableName);
ts = this.admin.getLastMajorCompactionTimestamp(tableName);
ADMIN.flush(tableName);
ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
// still 0, we flushed a file, but no major compaction happened
assertEquals(0, ts);
@ -461,30 +461,30 @@ public class TestAdmin1 {
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
regionName = l.getAllRegionLocations().get(0).getRegionInfo().getRegionName();
}
long ts1 = this.admin.getLastMajorCompactionTimestampForRegion(regionName);
long ts1 = ADMIN.getLastMajorCompactionTimestampForRegion(regionName);
assertEquals(ts, ts1);
p = new Put(Bytes.toBytes("row2"));
p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
table.put(p);
this.admin.flush(tableName);
ts = this.admin.getLastMajorCompactionTimestamp(tableName);
ADMIN.flush(tableName);
ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
// make sure the region API returns the same value, as the old file is still around
assertEquals(ts1, ts);
TEST_UTIL.compact(tableName, true);
table.put(p);
// forces a wait for the compaction
this.admin.flush(tableName);
ts = this.admin.getLastMajorCompactionTimestamp(tableName);
ADMIN.flush(tableName);
ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
// after a compaction our earliest timestamp will have progressed forward
assertTrue(ts > ts1);
// region api still the same
ts1 = this.admin.getLastMajorCompactionTimestampForRegion(regionName);
ts1 = ADMIN.getLastMajorCompactionTimestampForRegion(regionName);
assertEquals(ts, ts1);
table.put(p);
this.admin.flush(tableName);
ts = this.admin.getLastMajorCompactionTimestamp(tableName);
ADMIN.flush(tableName);
ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
assertEquals(ts, ts1);
table.close();
}
@ -509,18 +509,18 @@ public class TestAdmin1 {
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
// Make table read only
TableDescriptor htd = TableDescriptorBuilder.newBuilder(this.admin.getDescriptor(tableName))
TableDescriptor htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
.setReadOnly(true).build();
admin.modifyTable(htd);
ADMIN.modifyTable(htd);
// try to modify the read only table now
htd = TableDescriptorBuilder.newBuilder(this.admin.getDescriptor(tableName))
htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
.setCompactionEnabled(false).build();
admin.modifyTable(htd);
ADMIN.modifyTable(htd);
// Delete the table
this.admin.disableTable(tableName);
this.admin.deleteTable(tableName);
assertFalse(this.admin.tableExists(tableName));
ADMIN.disableTable(tableName);
ADMIN.deleteTable(tableName);
assertFalse(ADMIN.tableExists(tableName));
}
@Test(expected = TableNotDisabledException.class)
@ -529,17 +529,17 @@ public class TestAdmin1 {
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
// Modify region replication count
TableDescriptor htd = TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName))
TableDescriptor htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
.setRegionReplication(3).build();
try {
// try to modify the region replication count without disabling the table
admin.modifyTable(htd);
ADMIN.modifyTable(htd);
fail("Expected an exception");
} finally {
// Delete the table
admin.disableTable(tableName);
admin.deleteTable(tableName);
assertFalse(admin.tableExists(tableName));
ADMIN.disableTable(tableName);
ADMIN.deleteTable(tableName);
assertFalse(ADMIN.tableExists(tableName));
}
}
@ -549,14 +549,14 @@ public class TestAdmin1 {
@Test
public void testOnlineChangeTableSchema() throws IOException, InterruptedException {
final TableName tableName = TableName.valueOf(name.getMethodName());
HTableDescriptor [] tables = admin.listTables();
int numTables = tables.length;
List<TableDescriptor> tables = ADMIN.listTableDescriptors();
int numTables = tables.size();
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
tables = this.admin.listTables();
assertEquals(numTables + 1, tables.length);
tables = ADMIN.listTableDescriptors();
assertEquals(numTables + 1, tables.size());
// FIRST, do htabledescriptor changes.
HTableDescriptor htd = this.admin.getTableDescriptor(tableName);
HTableDescriptor htd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
// Make a copy and assert copy is good.
HTableDescriptor copy = new HTableDescriptor(htd);
assertTrue(htd.equals(copy));
@ -571,12 +571,12 @@ public class TestAdmin1 {
copy.setValue(key, key);
boolean expectedException = false;
try {
admin.modifyTable(tableName, copy);
ADMIN.modifyTable(copy);
} catch (TableNotDisabledException re) {
expectedException = true;
}
assertFalse(expectedException);
HTableDescriptor modifiedHtd = new HTableDescriptor(this.admin.getTableDescriptor(tableName));
HTableDescriptor modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
assertFalse(htd.equals(modifiedHtd));
assertTrue(copy.equals(modifiedHtd));
assertEquals(newFlushSize, modifiedHtd.getMemStoreFlushSize());
@ -592,44 +592,44 @@ public class TestAdmin1 {
final byte [] hcdName = hcd.getName();
expectedException = false;
try {
this.admin.modifyColumnFamily(tableName, hcd);
ADMIN.modifyColumnFamily(tableName, hcd);
} catch (TableNotDisabledException re) {
expectedException = true;
}
assertFalse(expectedException);
modifiedHtd = this.admin.getTableDescriptor(tableName);
modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
HColumnDescriptor modifiedHcd = modifiedHtd.getFamily(hcdName);
assertEquals(newMaxVersions, modifiedHcd.getMaxVersions());
// Try adding a column
assertFalse(this.admin.isTableDisabled(tableName));
assertFalse(ADMIN.isTableDisabled(tableName));
final String xtracolName = "xtracol";
HColumnDescriptor xtracol = new HColumnDescriptor(xtracolName);
xtracol.setValue(xtracolName, xtracolName);
expectedException = false;
try {
this.admin.addColumnFamily(tableName, xtracol);
ADMIN.addColumnFamily(tableName, xtracol);
} catch (TableNotDisabledException re) {
expectedException = true;
}
// Add column should work even if the table is enabled
assertFalse(expectedException);
modifiedHtd = this.admin.getTableDescriptor(tableName);
modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
hcd = modifiedHtd.getFamily(xtracol.getName());
assertTrue(hcd != null);
assertTrue(hcd.getValue(xtracolName).equals(xtracolName));
// Delete the just-added column.
this.admin.deleteColumnFamily(tableName, xtracol.getName());
modifiedHtd = this.admin.getTableDescriptor(tableName);
ADMIN.deleteColumnFamily(tableName, xtracol.getName());
modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
hcd = modifiedHtd.getFamily(xtracol.getName());
assertTrue(hcd == null);
// Delete the table
this.admin.disableTable(tableName);
this.admin.deleteTable(tableName);
this.admin.listTables();
assertFalse(this.admin.tableExists(tableName));
ADMIN.disableTable(tableName);
ADMIN.deleteTable(tableName);
ADMIN.listTableDescriptors();
assertFalse(ADMIN.tableExists(tableName));
}
protected void verifyRoundRobinDistribution(ClusterConnection c, RegionLocator regionLocator, int
@ -667,7 +667,7 @@ public class TestAdmin1 {
final TableName tableName = TableName.valueOf(name.getMethodName());
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc);
ADMIN.createTable(desc);
List<HRegionLocation> regions;
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
regions = l.getAllRegionLocations();
@ -677,7 +677,7 @@ public class TestAdmin1 {
TableName TABLE_2 = TableName.valueOf(tableName.getNameAsString() + "_2");
desc = new HTableDescriptor(TABLE_2);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, new byte[][]{new byte[]{42}});
ADMIN.createTable(desc, new byte[][]{new byte[]{42}});
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
regions = l.getAllRegionLocations();
assertEquals("Table should have only 2 region", 2, regions.size());
@ -686,7 +686,7 @@ public class TestAdmin1 {
TableName TABLE_3 = TableName.valueOf(tableName.getNameAsString() + "_3");
desc = new HTableDescriptor(TABLE_3);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 3);
ADMIN.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 3);
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
regions = l.getAllRegionLocations();
assertEquals("Table should have only 3 region", 3, regions.size());
@ -696,7 +696,7 @@ public class TestAdmin1 {
desc = new HTableDescriptor(TABLE_4);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
try {
admin.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 2);
ADMIN.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 2);
fail("Should not be able to create a table with only 2 regions using this API.");
} catch (IllegalArgumentException eae) {
// Expected
@ -705,7 +705,7 @@ public class TestAdmin1 {
TableName TABLE_5 = TableName.valueOf(tableName.getNameAsString() + "_5");
desc = new HTableDescriptor(TABLE_5);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16);
ADMIN.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16);
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_5)) {
regions = l.getAllRegionLocations();
assertEquals("Table should have 16 region", 16, regions.size());
@ -715,7 +715,6 @@ public class TestAdmin1 {
@Test
public void testCreateTableWithRegions() throws IOException, InterruptedException {
final TableName tableName = TableName.valueOf(name.getMethodName());
byte [][] splitKeys = {
new byte [] { 1, 1, 1 },
new byte [] { 2, 2, 2 },
@ -731,9 +730,9 @@ public class TestAdmin1 {
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, splitKeys);
ADMIN.createTable(desc, splitKeys);
boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys);
boolean tableAvailable = ADMIN.isTableAvailable(tableName);
assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable);
List<HRegionLocation> regions;
@ -799,8 +798,7 @@ public class TestAdmin1 {
desc = new HTableDescriptor(TABLE_2);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin = TEST_UTIL.getAdmin();
admin.createTable(desc, startKey, endKey, expectedRegions);
ADMIN.createTable(desc, startKey, endKey, expectedRegions);
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
regions = l.getAllRegionLocations();
@ -854,8 +852,7 @@ public class TestAdmin1 {
desc = new HTableDescriptor(TABLE_3);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin = TEST_UTIL.getAdmin();
admin.createTable(desc, startKey, endKey, expectedRegions);
ADMIN.createTable(desc, startKey, endKey, expectedRegions);
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
@ -880,7 +877,7 @@ public class TestAdmin1 {
desc = new HTableDescriptor(TABLE_4);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
try {
admin.createTable(desc, splitKeys);
ADMIN.createTable(desc, splitKeys);
assertTrue("Should not be able to create this table because of " +
"duplicate split keys", false);
} catch(IllegalArgumentException iae) {
@ -888,21 +885,6 @@ public class TestAdmin1 {
}
}
@Test
public void testTableAvailableWithRandomSplitKeys() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor("col"));
byte[][] splitKeys = new byte[1][];
splitKeys = new byte [][] {
new byte [] { 1, 1, 1 },
new byte [] { 2, 2, 2 }
};
admin.createTable(desc);
boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys);
assertFalse("Table should be created with 1 row in META", tableAvailable);
}
@Test
public void testCreateTableWithOnlyEmptyStartRow() throws IOException {
final byte[] tableName = Bytes.toBytes(name.getMethodName());
@ -911,7 +893,7 @@ public class TestAdmin1 {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
desc.addFamily(new HColumnDescriptor("col"));
try {
admin.createTable(desc, splitKeys);
ADMIN.createTable(desc, splitKeys);
fail("Test case should fail as empty split key is passed.");
} catch (IllegalArgumentException e) {
}
@ -927,7 +909,7 @@ public class TestAdmin1 {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
desc.addFamily(new HColumnDescriptor("col"));
try {
admin.createTable(desc, splitKeys);
ADMIN.createTable(desc, splitKeys);
fail("Test case should fail as empty split key is passed.");
} catch (IllegalArgumentException e) {
LOG.info("Expected ", e);
@ -938,10 +920,10 @@ public class TestAdmin1 {
public void testTableExist() throws IOException {
final TableName table = TableName.valueOf(name.getMethodName());
boolean exist;
exist = this.admin.tableExists(table);
exist = ADMIN.tableExists(table);
assertEquals(false, exist);
TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY);
exist = this.admin.tableExists(table);
exist = ADMIN.tableExists(table);
assertEquals(true, exist);
}
@ -979,7 +961,7 @@ public class TestAdmin1 {
int expectedRegions = splitKeys.length + 1;
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, splitKeys);
ADMIN.createTable(desc, splitKeys);
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
List<HRegionLocation> regions = l.getAllRegionLocations();
@ -988,9 +970,9 @@ public class TestAdmin1 {
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
expectedRegions, regions.size());
// Disable table.
admin.disableTable(tableName);
ADMIN.disableTable(tableName);
// Enable table, use retain assignment to assign regions.
admin.enableTable(tableName);
ADMIN.enableTable(tableName);
List<HRegionLocation> regions2 = l.getAllRegionLocations();
// Check the assignment.
@ -1041,7 +1023,7 @@ public class TestAdmin1 {
for (int i = 0; i < rowCounts.length; i++) {
sb.append("_").append(Integer.toString(rowCounts[i]));
}
assertFalse(admin.tableExists(tableName));
assertFalse(ADMIN.tableExists(tableName));
try (final Table table = TEST_UTIL.createTable(tableName, familyNames,
numVersions, blockSize);
final RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
@ -1089,7 +1071,7 @@ public class TestAdmin1 {
// Split the table
if (async) {
this.admin.split(tableName, splitPoint);
ADMIN.split(tableName, splitPoint);
final AtomicInteger count = new AtomicInteger(0);
Thread t = new Thread("CheckForSplit") {
@Override public void run() {
@ -1122,7 +1104,7 @@ public class TestAdmin1 {
t.join();
} else {
// Sync split region, no need to create a thread to check
((HBaseAdmin)admin).splitRegionSync(m.get(0).getRegionInfo().getRegionName(), splitPoint);
ADMIN.splitRegionAsync(m.get(0).getRegion().getRegionName(), splitPoint).get();
}
// Verify row count
@ -1214,7 +1196,7 @@ public class TestAdmin1 {
// the element at index 1 would be a replica (since the metareader gives us ordered
// regions). Try splitting that region via the split API . Should fail
try {
TEST_UTIL.getAdmin().splitRegion(regions.get(1).getFirst().getRegionName());
TEST_UTIL.getAdmin().splitRegionAsync(regions.get(1).getFirst().getRegionName(), null).get();
} catch (IllegalArgumentException ex) {
gotException = true;
}
@ -1287,54 +1269,54 @@ public class TestAdmin1 {
public void testEnableDisableAddColumnDeleteColumn() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
while (!this.admin.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
while (!ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
Thread.sleep(10);
}
this.admin.disableTable(tableName);
ADMIN.disableTable(tableName);
try {
TEST_UTIL.getConnection().getTable(tableName);
} catch (org.apache.hadoop.hbase.DoNotRetryIOException e) {
//expected
}
this.admin.addColumnFamily(tableName, new HColumnDescriptor("col2"));
this.admin.enableTable(tableName);
ADMIN.addColumnFamily(tableName, new HColumnDescriptor("col2"));
ADMIN.enableTable(tableName);
try {
this.admin.deleteColumnFamily(tableName, Bytes.toBytes("col2"));
ADMIN.deleteColumnFamily(tableName, Bytes.toBytes("col2"));
} catch (TableNotDisabledException e) {
LOG.info(e.toString(), e);
}
this.admin.disableTable(tableName);
this.admin.deleteTable(tableName);
ADMIN.disableTable(tableName);
ADMIN.deleteTable(tableName);
}
@Test
public void testDeleteLastColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
while (!this.admin.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
while (!ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
Thread.sleep(10);
}
// test for enabled table
try {
this.admin.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
ADMIN.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
fail("Should have failed to delete the only column family of a table");
} catch (InvalidFamilyOperationException ex) {
// expected
}
// test for disabled table
this.admin.disableTable(tableName);
ADMIN.disableTable(tableName);
try {
this.admin.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
ADMIN.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
fail("Should have failed to delete the only column family of a table");
} catch (InvalidFamilyOperationException ex) {
// expected
}
this.admin.deleteTable(tableName);
ADMIN.deleteTable(tableName);
}
/*
@ -1365,7 +1347,7 @@ public class TestAdmin1 {
puts.add(p);
try {
table.put(puts);
admin.flush(tableName);
ADMIN.flush(tableName);
List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(tableName);
for (HRegion r : regions) {
@ -1385,9 +1367,9 @@ public class TestAdmin1 {
}
}
} finally {
if (admin.isTableEnabled(tableName)) {
this.admin.disableTable(tableName);
this.admin.deleteTable(tableName);
if (ADMIN.isTableEnabled(tableName)) {
ADMIN.disableTable(tableName);
ADMIN.deleteTable(tableName);
}
}
}
@ -1410,29 +1392,29 @@ public class TestAdmin1 {
RegionInfo regionB;
// merge with full name
tableRegions = admin.getRegions(tableName);
assertEquals(3, admin.getTableRegions(tableName).size());
tableRegions = ADMIN.getRegions(tableName);
assertEquals(3, ADMIN.getRegions(tableName).size());
regionA = tableRegions.get(0);
regionB = tableRegions.get(1);
// TODO convert this to version that is synchronous (See HBASE-16668)
admin.mergeRegionsAsync(regionA.getRegionName(), regionB.getRegionName(), false)
ADMIN.mergeRegionsAsync(regionA.getRegionName(), regionB.getRegionName(), false)
.get(60, TimeUnit.SECONDS);
assertEquals(2, admin.getTableRegions(tableName).size());
assertEquals(2, ADMIN.getRegions(tableName).size());
// merge with encoded name
tableRegions = admin.getRegions(tableName);
tableRegions = ADMIN.getRegions(tableName);
regionA = tableRegions.get(0);
regionB = tableRegions.get(1);
// TODO convert this to version that is synchronous (See HBASE-16668)
admin.mergeRegionsAsync(
ADMIN.mergeRegionsAsync(
regionA.getEncodedNameAsBytes(), regionB.getEncodedNameAsBytes(), false)
.get(60, TimeUnit.SECONDS);
assertEquals(1, admin.getTableRegions(tableName).size());
assertEquals(1, ADMIN.getRegions(tableName).size());
} finally {
this.admin.disableTable(tableName);
this.admin.deleteTable(tableName);
ADMIN.disableTable(tableName);
ADMIN.deleteTable(tableName);
}
}
@ -1451,16 +1433,16 @@ public class TestAdmin1 {
p.addColumn(Bytes.toBytes("f"), q1, v1);
table.put(p);
}
this.admin.flush(tableName);
ADMIN.flush(tableName);
try {
this.admin.split(tableName, Bytes.toBytes("row5"));
ADMIN.split(tableName, Bytes.toBytes("row5"));
Threads.sleep(10000);
} catch (Exception e) {
// Nothing to do.
}
// Split should not happen.
List<RegionInfo> allRegions = MetaTableAccessor.getTableRegions(
this.admin.getConnection(), tableName, true);
ADMIN.getConnection(), tableName, true);
assertEquals(1, allRegions.size());
}
@ -1502,15 +1484,15 @@ public class TestAdmin1 {
.setTimeToLive(TTL)
.build()
).build();
admin.createTable(tableDesc, splitKeys);
ADMIN.createTable(tableDesc, splitKeys);
assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
assertTrue("Table should be created with splitKyes + 1 rows in META",
admin.isTableAvailable(tableName, splitKeys));
ADMIN.isTableAvailable(tableName));
// clone & Verify
admin.cloneTableSchema(tableName, newTableName, preserveSplits);
TableDescriptor newTableDesc = admin.getDescriptor(newTableName);
ADMIN.cloneTableSchema(tableName, newTableName, preserveSplits);
TableDescriptor newTableDesc = ADMIN.getDescriptor(newTableName);
assertEquals(NUM_FAMILYS, newTableDesc.getColumnFamilyCount());
assertEquals(BLOCK_SIZE, newTableDesc.getColumnFamily(FAMILY_1).getBlocksize());
@ -1521,7 +1503,7 @@ public class TestAdmin1 {
if (preserveSplits) {
assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
assertTrue("New table should be created with splitKyes + 1 rows in META",
admin.isTableAvailable(newTableName, splitKeys));
ADMIN.isTableAvailable(newTableName));
} else {
assertEquals(1, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
}
@ -1533,7 +1515,7 @@ public class TestAdmin1 {
final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
// test for non-existent source table
try {
admin.cloneTableSchema(tableName, newTableName, false);
ADMIN.cloneTableSchema(tableName, newTableName, false);
fail("Should have failed to create a new table by cloning non-existent source table.");
} catch (TableNotFoundException ex) {
// expected
@ -1549,7 +1531,7 @@ public class TestAdmin1 {
TEST_UTIL.createTable(newTableName, FAMILY_0);
// test for existent destination table
try {
admin.cloneTableSchema(tableName, newTableName, false);
ADMIN.cloneTableSchema(tableName, newTableName, false);
fail("Should have failed to create a existent table.");
} catch (TableExistsException ex) {
// expected
@ -1564,15 +1546,15 @@ public class TestAdmin1 {
.setRegionReplication(5)
.build();
admin.createTable(desc);
ADMIN.createTable(desc);
int maxFileSize = 10000000;
TableDescriptor newDesc = TableDescriptorBuilder.newBuilder(desc)
.setMaxFileSize(maxFileSize)
.build();
admin.modifyTable(newDesc);
TableDescriptor newTableDesc = admin.getDescriptor(tableName);
ADMIN.modifyTable(newDesc);
TableDescriptor newTableDesc = ADMIN.getDescriptor(tableName);
assertEquals(maxFileSize, newTableDesc.getMaxFileSize());
}
}

View File

@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Rule;
@ -89,7 +88,7 @@ public class TestAdmin2 {
private static final Logger LOG = LoggerFactory.getLogger(TestAdmin2.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private Admin admin;
private static Admin ADMIN;
@Rule
public TestName name = new TestName();
@ -103,6 +102,7 @@ public class TestAdmin2 {
TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 30);
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
TEST_UTIL.startMiniCluster(3);
ADMIN = TEST_UTIL.getAdmin();
}
@AfterClass
@ -110,14 +110,9 @@ public class TestAdmin2 {
TEST_UTIL.shutdownMiniCluster();
}
@Before
public void setUp() throws Exception {
this.admin = TEST_UTIL.getHBaseAdmin();
}
@After
public void tearDown() throws Exception {
for (HTableDescriptor htd : this.admin.listTables()) {
for (TableDescriptor htd : ADMIN.listTableDescriptors()) {
TEST_UTIL.deleteTable(htd.getTableName());
}
}
@ -126,7 +121,7 @@ public class TestAdmin2 {
public void testCreateBadTables() throws IOException {
String msg = null;
try {
this.admin.createTable(new HTableDescriptor(TableName.META_TABLE_NAME));
ADMIN.createTable(new HTableDescriptor(TableName.META_TABLE_NAME));
} catch(TableExistsException e) {
msg = e.toString();
}
@ -141,7 +136,7 @@ public class TestAdmin2 {
Thread [] threads = new Thread [count];
final AtomicInteger successes = new AtomicInteger(0);
final AtomicInteger failures = new AtomicInteger(0);
final Admin localAdmin = this.admin;
final Admin localAdmin = ADMIN;
for (int i = 0; i < count; i++) {
threads[i] = new Thread(Integer.toString(i)) {
@Override
@ -186,8 +181,8 @@ public class TestAdmin2 {
HTableDescriptor htd2 = new HTableDescriptor(TableName.valueOf(name));
htd1.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
htd2.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(htd1);
admin.createTable(htd2);
ADMIN.createTable(htd1);
ADMIN.createTable(htd2);
// Before fix, below would fail throwing a NoServerForRegionException.
TEST_UTIL.getConnection().getTable(htd2.getTableName()).close();
}
@ -197,7 +192,6 @@ public class TestAdmin2 {
* Thus creating of table with lots of regions can cause RPC timeout
* After the fix to make createTable truly async, RPC timeout shouldn't be an
* issue anymore
* @throws Exception
*/
@Test
public void testCreateTableRPCTimeOut() throws Exception {
@ -238,7 +232,6 @@ public class TestAdmin2 {
/**
* Test that user table names can contain '-' and '.' so long as they do not
* start with same. HBASE-771
* @throws IOException
*/
@Test
public void testTableNames() throws IOException {
@ -267,7 +260,6 @@ public class TestAdmin2 {
/**
* For HADOOP-2579
* @throws IOException
*/
@Test (expected=TableExistsException.class)
public void testTableExistsExceptionWithATable() throws IOException {
@ -278,41 +270,35 @@ public class TestAdmin2 {
/**
* Can't disable a table if the table isn't in enabled state
* @throws IOException
*/
@Test (expected=TableNotEnabledException.class)
public void testTableNotEnabledExceptionWithATable() throws IOException {
final TableName name = TableName.valueOf(this.name.getMethodName());
TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY).close();
this.admin.disableTable(name);
this.admin.disableTable(name);
ADMIN.disableTable(name);
ADMIN.disableTable(name);
}
/**
* Can't enable a table if the table isn't in disabled state
* @throws IOException
*/
@Test (expected=TableNotDisabledException.class)
public void testTableNotDisabledExceptionWithATable() throws IOException {
final TableName name = TableName.valueOf(this.name.getMethodName());
Table t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
try {
this.admin.enableTable(name);
}finally {
t.close();
try (Table t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY)) {
ADMIN.enableTable(name);
}
}
/**
* For HADOOP-2579
* @throws IOException
*/
@Test (expected=TableNotFoundException.class)
@Test(expected = TableNotFoundException.class)
public void testTableNotFoundExceptionWithoutAnyTables() throws IOException {
TableName tableName = TableName
.valueOf("testTableNotFoundExceptionWithoutAnyTables");
Table ht = TEST_UTIL.getConnection().getTable(tableName);
ht.get(new Get(Bytes.toBytes("e")));
TableName tableName = TableName.valueOf("testTableNotFoundExceptionWithoutAnyTables");
try (Table ht = TEST_UTIL.getConnection().getTable(tableName)) {
ht.get(new Get(Bytes.toBytes("e")));
}
}
@Test
@ -326,7 +312,7 @@ public class TestAdmin2 {
for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.getTable().isSystemTable()) {
info = regionInfo;
admin.unassign(regionInfo.getRegionName(), true);
ADMIN.unassign(regionInfo.getRegionName(), true);
}
}
boolean isInList = ProtobufUtil.getOnlineRegions(
@ -356,7 +342,7 @@ public class TestAdmin2 {
if (regionInfo.getRegionNameAsString().contains(name)) {
info = regionInfo;
try {
admin.unassign(Bytes.toBytes("sample"), true);
ADMIN.unassign(Bytes.toBytes("sample"), true);
} catch (UnknownRegionException nsre) {
// expected, ignore it
}
@ -380,7 +366,7 @@ public class TestAdmin2 {
if (!regionInfo.isMetaRegion()) {
if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion2")) {
info = regionInfo;
admin.unassign(regionInfo.getRegionName(), true);
ADMIN.unassign(regionInfo.getRegionName(), true);
}
}
}
@ -418,7 +404,7 @@ public class TestAdmin2 {
HColumnDescriptor hcd = new HColumnDescriptor("value");
htd.addFamily(hcd);
admin.createTable(htd, null);
ADMIN.createTable(htd, null);
}
/**
@ -438,9 +424,9 @@ public class TestAdmin2 {
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, startKey, endKey, expectedRegions);
ADMIN.createTable(desc, startKey, endKey, expectedRegions);
List<RegionInfo> RegionInfos = admin.getRegions(tableName);
List<RegionInfo> RegionInfos = ADMIN.getRegions(tableName);
assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + RegionInfos.size(),
@ -479,7 +465,7 @@ public class TestAdmin2 {
for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
r.flush(true);
}
admin.rollWALWriter(regionServer.getServerName());
ADMIN.rollWALWriter(regionServer.getServerName());
int count = AbstractFSWALProvider.getNumRolledLogFiles(regionServer.getWAL(null));
LOG.info("after flushing all regions and rolling logs there are " +
count + " log files");
@ -536,7 +522,7 @@ public class TestAdmin2 {
// Create the test table and open it
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc);
ADMIN.createTable(desc);
Table table = TEST_UTIL.getConnection().getTable(tableName);
HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
@ -585,7 +571,7 @@ public class TestAdmin2 {
@Test
public void testDisableCatalogTable() throws Exception {
try {
this.admin.disableTable(TableName.META_TABLE_NAME);
ADMIN.disableTable(TableName.META_TABLE_NAME);
fail("Expected to throw ConstraintException");
} catch (ConstraintException e) {
}
@ -601,13 +587,13 @@ public class TestAdmin2 {
@Test
public void testIsEnabledOrDisabledOnUnknownTable() throws Exception {
try {
admin.isTableEnabled(TableName.valueOf(name.getMethodName()));
ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()));
fail("Test should fail if isTableEnabled called on unknown table.");
} catch (IOException e) {
}
try {
admin.isTableDisabled(TableName.valueOf(name.getMethodName()));
ADMIN.isTableDisabled(TableName.valueOf(name.getMethodName()));
fail("Test should fail if isTableDisabled called on unknown table.");
} catch (IOException e) {
}
@ -636,46 +622,46 @@ public class TestAdmin2 {
@Test
public void testBalancer() throws Exception {
boolean initialState = admin.isBalancerEnabled();
boolean initialState = ADMIN.isBalancerEnabled();
// Start the balancer, wait for it.
boolean prevState = admin.setBalancerRunning(!initialState, true);
boolean prevState = ADMIN.balancerSwitch(!initialState, true);
// The previous state should be the original state we observed
assertEquals(initialState, prevState);
// Current state should be opposite of the original
assertEquals(!initialState, admin.isBalancerEnabled());
assertEquals(!initialState, ADMIN.isBalancerEnabled());
// Reset it back to what it was
prevState = admin.setBalancerRunning(initialState, true);
prevState = ADMIN.balancerSwitch(initialState, true);
// The previous state should be the opposite of the initial state
assertEquals(!initialState, prevState);
// Current state should be the original state again
assertEquals(initialState, admin.isBalancerEnabled());
assertEquals(initialState, ADMIN.isBalancerEnabled());
}
@Test
public void testRegionNormalizer() throws Exception {
boolean initialState = admin.isNormalizerEnabled();
boolean initialState = ADMIN.isNormalizerEnabled();
// flip state
boolean prevState = admin.setNormalizerRunning(!initialState);
boolean prevState = ADMIN.normalizerSwitch(!initialState);
// The previous state should be the original state we observed
assertEquals(initialState, prevState);
// Current state should be opposite of the original
assertEquals(!initialState, admin.isNormalizerEnabled());
assertEquals(!initialState, ADMIN.isNormalizerEnabled());
// Reset it back to what it was
prevState = admin.setNormalizerRunning(initialState);
prevState = ADMIN.normalizerSwitch(initialState);
// The previous state should be the opposite of the initial state
assertEquals(!initialState, prevState);
// Current state should be the original state again
assertEquals(initialState, admin.isNormalizerEnabled());
assertEquals(initialState, ADMIN.isNormalizerEnabled());
}
@Test
@ -683,32 +669,32 @@ public class TestAdmin2 {
Random randomGenerator = new Random();
long procId = randomGenerator.nextLong();
boolean abortResult = admin.abortProcedure(procId, true);
boolean abortResult = ADMIN.abortProcedure(procId, true);
assertFalse(abortResult);
}
@Test
public void testGetProcedures() throws Exception {
String procList = admin.getProcedures();
String procList = ADMIN.getProcedures();
assertTrue(procList.startsWith("["));
}
@Test
public void testGetLocks() throws Exception {
String lockList = admin.getLocks();
String lockList = ADMIN.getLocks();
assertTrue(lockList.startsWith("["));
}
@Test
public void testDecommissionRegionServers() throws Exception {
List<ServerName> decommissionedRegionServers = admin.listDecommissionedRegionServers();
List<ServerName> decommissionedRegionServers = ADMIN.listDecommissionedRegionServers();
assertTrue(decommissionedRegionServers.isEmpty());
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.createMultiRegionTable(tableName, Bytes.toBytes("f"), 6);
ArrayList<ServerName> clusterRegionServers =
new ArrayList<>(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
new ArrayList<>(ADMIN.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
.getLiveServerMetrics().keySet());
assertEquals(3, clusterRegionServers.size());
@ -718,8 +704,8 @@ public class TestAdmin2 {
// leaving one online.
int i;
for (i = 0; i < clusterRegionServers.size(); i++) {
List<RegionInfo> regionsOnServer = admin.getRegions(clusterRegionServers.get(i));
if (admin.getRegions(clusterRegionServers.get(i)).stream().anyMatch(p -> p.isMetaRegion())) {
List<RegionInfo> regionsOnServer = ADMIN.getRegions(clusterRegionServers.get(i));
if (ADMIN.getRegions(clusterRegionServers.get(i)).stream().anyMatch(p -> p.isMetaRegion())) {
serversToDecommssion.put(clusterRegionServers.get(i), regionsOnServer);
break;
}
@ -728,13 +714,13 @@ public class TestAdmin2 {
clusterRegionServers.remove(i);
// Get another server to decommission.
serversToDecommssion.put(clusterRegionServers.get(0),
admin.getRegions(clusterRegionServers.get(0)));
ADMIN.getRegions(clusterRegionServers.get(0)));
ServerName remainingServer = clusterRegionServers.get(1);
// Decommission
admin.decommissionRegionServers(new ArrayList<ServerName>(serversToDecommssion.keySet()), true);
assertEquals(2, admin.listDecommissionedRegionServers().size());
ADMIN.decommissionRegionServers(new ArrayList<ServerName>(serversToDecommssion.keySet()), true);
assertEquals(2, ADMIN.listDecommissionedRegionServers().size());
// Verify the regions have been off the decommissioned servers, all on the one
// remaining server.
@ -748,9 +734,9 @@ public class TestAdmin2 {
for (ServerName server : serversToDecommssion.keySet()) {
List<byte[]> encodedRegionNames = serversToDecommssion.get(server).stream()
.map(region -> region.getEncodedNameAsBytes()).collect(Collectors.toList());
admin.recommissionRegionServer(server, encodedRegionNames);
ADMIN.recommissionRegionServer(server, encodedRegionNames);
}
assertTrue(admin.listDecommissionedRegionServers().isEmpty());
assertTrue(ADMIN.listDecommissionedRegionServers().isEmpty());
// Verify the regions have been moved to the recommissioned servers
for (ServerName server : serversToDecommssion.keySet()) {
for (RegionInfo region : serversToDecommssion.get(server)) {
@ -770,7 +756,7 @@ public class TestAdmin2 {
table.put(new Put(Bytes.toBytes(i)).addColumn(Bytes.toBytes("f"), Bytes.toBytes("q"),
Bytes.toBytes(i)));
}
admin.flush(tableName);
ADMIN.flush(tableName);
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
List<HRegion> regions = rs.getRegions(tableName);
@ -783,7 +769,7 @@ public class TestAdmin2 {
Assert.assertNotNull(store);
Assert.assertEquals(expectedStoreFilesSize, store.getSize());
ClusterConnection conn = ((ClusterConnection) admin.getConnection());
ClusterConnection conn = ((ClusterConnection) ADMIN.getConnection());
HBaseRpcController controller = conn.getRpcControllerFactory().newController();
for (int i = 0; i < 10; i++) {
RegionInfo ri =
@ -801,26 +787,26 @@ public class TestAdmin2 {
TEST_UTIL.createTable(tableName, Bytes.toBytes("f"));
// get the original table region count
List<RegionInfo> regions = admin.getRegions(tableName);
List<RegionInfo> regions = ADMIN.getRegions(tableName);
int originalCount = regions.size();
assertEquals(1, originalCount);
// split the table and wait until region count increases
admin.split(tableName, Bytes.toBytes(3));
ADMIN.split(tableName, Bytes.toBytes(3));
TEST_UTIL.waitFor(30000, new Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return admin.getRegions(tableName).size() > originalCount;
return ADMIN.getRegions(tableName).size() > originalCount;
}
});
// do some table modification
TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName))
TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
.setMaxFileSize(11111111)
.build();
admin.modifyTable(tableDesc);
assertEquals(11111111, admin.getDescriptor(tableName).getMaxFileSize());
ADMIN.modifyTable(tableDesc);
assertEquals(11111111, ADMIN.getDescriptor(tableName).getMaxFileSize());
}
@Test
@ -830,7 +816,7 @@ public class TestAdmin2 {
new byte[][] { Bytes.toBytes(3) });
// assert we have at least 2 regions in the table
List<RegionInfo> regions = admin.getRegions(tableName);
List<RegionInfo> regions = ADMIN.getRegions(tableName);
int originalCount = regions.size();
assertTrue(originalCount >= 2);
@ -838,20 +824,20 @@ public class TestAdmin2 {
byte[] nameOfRegionB = regions.get(1).getEncodedNameAsBytes();
// merge the table regions and wait until region count decreases
admin.mergeRegionsAsync(nameOfRegionA, nameOfRegionB, true);
ADMIN.mergeRegionsAsync(nameOfRegionA, nameOfRegionB, true);
TEST_UTIL.waitFor(30000, new Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return admin.getRegions(tableName).size() < originalCount;
return ADMIN.getRegions(tableName).size() < originalCount;
}
});
// do some table modification
TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName))
TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
.setMaxFileSize(11111111)
.build();
admin.modifyTable(tableDesc);
assertEquals(11111111, admin.getDescriptor(tableName).getMaxFileSize());
ADMIN.modifyTable(tableDesc);
assertEquals(11111111, ADMIN.getDescriptor(tableName).getMaxFileSize());
}
}

View File

@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@ -61,7 +60,7 @@ public class TestAlwaysSetScannerId {
private static final int COUNT = 10;
private static HRegionInfo HRI;
private static RegionInfo HRI;
private static ClientProtos.ClientService.BlockingInterface STUB;
@ -73,7 +72,7 @@ public class TestAlwaysSetScannerId {
table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
}
}
HRI = UTIL.getAdmin().getTableRegions(TABLE_NAME).get(0);
HRI = UTIL.getAdmin().getRegions(TABLE_NAME).get(0);
STUB = ((ConnectionImplementation) UTIL.getConnection())
.getClient(UTIL.getHBaseCluster().getRegionServer(0).getServerName());
}

View File

@ -106,7 +106,7 @@ public class TestClientTimeouts {
admin = connection.getAdmin();
// run some admin commands
HBaseAdmin.available(conf);
admin.setBalancerRunning(false, false);
admin.balancerSwitch(false, false);
} catch (MasterNotRunningException ex) {
// Since we are randomly throwing SocketTimeoutExceptions, it is possible to get
// a MasterNotRunningException. It's a bug if we get other exceptions.

View File

@ -284,7 +284,7 @@ public class TestConnectionImplementation {
TableName tableName = TableName.valueOf("HCM-testConnectionClose" + allowsInterrupt);
TEST_UTIL.createTable(tableName, FAM_NAM).close();
boolean previousBalance = TEST_UTIL.getAdmin().setBalancerRunning(false, true);
boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true);
Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
// We want to work on a separate connection.
@ -366,7 +366,7 @@ public class TestConnectionImplementation {
table.close();
connection.close();
Assert.assertTrue("Unexpected exception is " + failed.get(), failed.get() == null);
TEST_UTIL.getAdmin().setBalancerRunning(previousBalance, true);
TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true);
}
/**
@ -377,7 +377,7 @@ public class TestConnectionImplementation {
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.createTable(tableName, FAM_NAM).close();
int idleTime = 20000;
boolean previousBalance = TEST_UTIL.getAdmin().setBalancerRunning(false, true);
boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true);
Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
// We want to work on a separate connection.
@ -425,7 +425,7 @@ public class TestConnectionImplementation {
connection.close();
EnvironmentEdgeManager.reset();
TEST_UTIL.getAdmin().setBalancerRunning(previousBalance, true);
TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true);
}
/**
@ -438,7 +438,7 @@ public class TestConnectionImplementation {
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.createTable(tableName, FAM_NAM).close();
boolean previousBalance = TEST_UTIL.getAdmin().setBalancerRunning(false, true);
boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true);
Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
// We want to work on a separate connection.
@ -493,7 +493,7 @@ public class TestConnectionImplementation {
} finally {
syncBlockingFilter.set(true);
t.join();
TEST_UTIL.getAdmin().setBalancerRunning(previousBalance, true);
TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true);
}
table.close();
@ -572,7 +572,7 @@ public class TestConnectionImplementation {
assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
assertNotNull(conn.getCachedLocation(TableName.valueOf(TABLE_NAME.getName()), ROW.clone()));
TEST_UTIL.getAdmin().setBalancerRunning(false, false);
TEST_UTIL.getAdmin().balancerSwitch(false, false);
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
// We can wait for all regions to be online, that makes log reading easier when debugging
@ -874,7 +874,7 @@ public class TestConnectionImplementation {
conn.clearRegionCache(TABLE_NAME3);
Assert.assertEquals(0, conn.getNumberOfCachedRegionLocations(TABLE_NAME3));
TEST_UTIL.getAdmin().setBalancerRunning(false, false);
TEST_UTIL.getAdmin().balancerSwitch(false, false);
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
// We can wait for all regions to be online, that makes log reading easier when debugging

View File

@ -29,7 +29,6 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
@ -4211,15 +4210,14 @@ public class TestFromClientSide {
TEST_UTIL.createTable(tables[i], FAMILY);
}
Admin admin = TEST_UTIL.getAdmin();
HTableDescriptor[] ts = admin.listTables();
HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length);
Collections.addAll(result, ts);
List<TableDescriptor> ts = admin.listTableDescriptors();
HashSet<TableDescriptor> result = new HashSet<>(ts);
int size = result.size();
assertTrue(size >= tables.length);
for (int i = 0; i < tables.length && i < size; i++) {
boolean found = false;
for (int j = 0; j < ts.length; j++) {
if (ts[j].getTableName().equals(tables[i])) {
for (int j = 0; j < ts.size(); j++) {
if (ts.get(j).getTableName().equals(tables[i])) {
found = true;
break;
}
@ -4334,7 +4332,7 @@ public class TestFromClientSide {
for (HColumnDescriptor c : desc.getFamilies())
c.setValue(attrName, attrValue);
// update metadata for all regions of this table
admin.modifyTable(tableAname, desc);
admin.modifyTable(desc);
// enable the table
admin.enableTable(tableAname);
@ -6364,7 +6362,7 @@ public class TestFromClientSide {
byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE;
Admin admin = TEST_UTIL.getAdmin();
admin.createTable(htd, KEYS);
List<HRegionInfo> regions = admin.getTableRegions(htd.getTableName());
List<RegionInfo> regions = admin.getRegions(htd.getTableName());
HRegionLocator locator =
(HRegionLocator) admin.getConnection().getRegionLocator(htd.getTableName());
@ -6372,7 +6370,7 @@ public class TestFromClientSide {
List<RegionLocations> regionLocations = new ArrayList<>();
// mock region locations coming from meta with multiple replicas
for (HRegionInfo region : regions) {
for (RegionInfo region : regions) {
HRegionLocation[] arr = new HRegionLocation[regionReplication];
for (int i = 0; i < arr.length; i++) {
arr[i] = new HRegionLocation(RegionReplicaUtil.getRegionInfoForReplica(region, i), null);

View File

@ -64,7 +64,6 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
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.Pair;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@ -121,23 +120,17 @@ public class TestFromClientSide3 {
TEST_UTIL.shutdownMiniCluster();
}
/**
* @throws java.lang.Exception
*/
@Before
public void setUp() throws Exception {
// Nothing to do.
}
/**
* @throws java.lang.Exception
*/
@After
public void tearDown() throws Exception {
for (HTableDescriptor htd: TEST_UTIL.getAdmin().listTables()) {
for (TableDescriptor htd : TEST_UTIL.getAdmin().listTableDescriptors()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
TEST_UTIL.deleteTable(htd.getTableName());
}
}
}
private void randomCFPuts(Table table, byte[] row, byte[] family, int nPuts)
@ -325,12 +318,7 @@ public class TestFromClientSide3 {
LOG.info("hbase.hstore.compaction.min should now be 5");
HTableDescriptor htd = new HTableDescriptor(hTable.getTableDescriptor());
htd.setValue("hbase.hstore.compaction.min", String.valueOf(5));
admin.modifyTable(tableName, htd);
Pair<Integer, Integer> st;
while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
LOG.debug(st.getFirst() + " regions left to update");
Thread.sleep(40);
}
admin.modifyTable(htd);
LOG.info("alter status finished");
// Create 3 more store files.
@ -352,11 +340,7 @@ public class TestFromClientSide3 {
HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2));
htd.modifyFamily(hcd);
admin.modifyTable(tableName, htd);
while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
LOG.debug(st.getFirst() + " regions left to update");
Thread.sleep(40);
}
admin.modifyTable(htd);
LOG.info("alter status finished");
// Issue a compaction request
@ -387,11 +371,7 @@ public class TestFromClientSide3 {
hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
hcd.setValue("hbase.hstore.compaction.min", null);
htd.modifyFamily(hcd);
admin.modifyTable(tableName, htd);
while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
LOG.debug(st.getFirst() + " regions left to update");
Thread.sleep(40);
}
admin.modifyTable(htd);
LOG.info("alter status finished");
assertNull(hTable.getTableDescriptor().getFamily(FAMILY).getValue(
"hbase.hstore.compaction.min"));

View File

@ -129,7 +129,7 @@ public class TestHBaseAdminNoCluster {
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.listTables();
admin.listTableDescriptors();
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@ -153,11 +153,11 @@ public class TestHBaseAdminNoCluster {
}
});
// Admin.getTableDescriptor()
// Admin.getDescriptor()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.getTableDescriptor(TableName.valueOf(name.getMethodName()));
admin.getDescriptor(TableName.valueOf(name.getMethodName()));
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@ -171,7 +171,7 @@ public class TestHBaseAdminNoCluster {
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.getTableDescriptorsByTableName(new ArrayList<>());
admin.listTableDescriptors(new ArrayList<>());
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@ -209,11 +209,11 @@ public class TestHBaseAdminNoCluster {
}
});
// Admin.setBalancerRunning()
// Admin.balancerSwitch()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.setBalancerRunning(true, true);
admin.balancerSwitch(true, true);
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@ -223,11 +223,11 @@ public class TestHBaseAdminNoCluster {
}
});
// Admin.balancer()
// Admin.balance()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.balancer();
admin.balance();
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@ -241,7 +241,7 @@ public class TestHBaseAdminNoCluster {
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.enableCatalogJanitor(true);
admin.catalogJanitorSwitch(true);
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@ -255,7 +255,7 @@ public class TestHBaseAdminNoCluster {
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.runCatalogScan();
admin.runCatalogJanitor();
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {

View File

@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.CompatibilityFactory;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.MetricsHBaseServerSource;
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
@ -84,20 +83,14 @@ public class TestLeaseRenewal {
TEST_UTIL.shutdownMiniCluster();
}
/**
* @throws java.lang.Exception
*/
@Before
public void setUp() throws Exception {
// Nothing to do.
}
/**
* @throws java.lang.Exception
*/
@After
public void tearDown() throws Exception {
for (HTableDescriptor htd : TEST_UTIL.getAdmin().listTables()) {
for (TableDescriptor htd : TEST_UTIL.getAdmin().listTableDescriptors()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
TEST_UTIL.deleteTable(htd.getTableName());
}

View File

@ -157,7 +157,7 @@ public class TestMetaWithReplicas {
@Test
public void testMetaHTDReplicaCount() throws Exception {
assertTrue(TEST_UTIL.getAdmin().getTableDescriptor(TableName.META_TABLE_NAME)
assertTrue(TEST_UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME)
.getRegionReplication() == 3);
}

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.Optional;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@ -112,14 +113,14 @@ public class TestMobCloneSnapshotFromClientCloneLinksAfterDelete
delayFlush = true;
SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 20, FAMILY);
long tid = System.currentTimeMillis();
byte[] snapshotName3 = Bytes.toBytes("snaptb3-" + tid);
String snapshotName3 = "snaptb3-" + tid;
TableName clonedTableName3 =
TableName.valueOf(name.getMethodName() + System.currentTimeMillis());
admin.snapshot(snapshotName3, tableName);
delayFlush = false;
int snapshot3Rows = -1;
try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
snapshot3Rows = TEST_UTIL.countRows(table);
snapshot3Rows = HBaseTestingUtility.countRows(table);
}
admin.cloneSnapshot(snapshotName3, clonedTableName3);
admin.deleteSnapshot(snapshotName3);

View File

@ -96,7 +96,7 @@ public class TestMultiRespectsLimits {
TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return admin.getTableRegions(tableName).size() > 1;
return admin.getRegions(tableName).size() > 1;
}
});
}

View File

@ -670,7 +670,7 @@ public class TestReplicaWithCluster {
// within configured hbase.client.metaReplicaCallTimeout.scan from primary meta region.
@Test
public void testGetRegionLocationFromPrimaryMetaRegion() throws IOException, InterruptedException {
HTU.getAdmin().setBalancerRunning(false, true);
HTU.getAdmin().balancerSwitch(false, true);
((ConnectionImplementation) HTU.getAdmin().getConnection()).setUseMetaReplicas(true);
@ -690,7 +690,7 @@ public class TestReplicaWithCluster {
} finally {
RegionServerHostingPrimayMetaRegionSlowOrStopCopro.slowDownPrimaryMetaScan = false;
((ConnectionImplementation) HTU.getAdmin().getConnection()).setUseMetaReplicas(false);
HTU.getAdmin().setBalancerRunning(true, true);
HTU.getAdmin().balancerSwitch(true, true);
HTU.getAdmin().disableTable(hdt.getTableName());
HTU.deleteTable(hdt.getTableName());
}
@ -703,7 +703,7 @@ public class TestReplicaWithCluster {
// with the primary meta region.
@Test
public void testReplicaGetWithPrimaryAndMetaDown() throws IOException, InterruptedException {
HTU.getAdmin().setBalancerRunning(false, true);
HTU.getAdmin().balancerSwitch(false, true);
((ConnectionImplementation)HTU.getAdmin().getConnection()).setUseMetaReplicas(true);
@ -789,7 +789,7 @@ public class TestReplicaWithCluster {
} finally {
((ConnectionImplementation)HTU.getAdmin().getConnection()).setUseMetaReplicas(false);
RegionServerHostingPrimayMetaRegionSlowOrStopCopro.throwException = false;
HTU.getAdmin().setBalancerRunning(true, true);
HTU.getAdmin().balancerSwitch(true, true);
HTU.getAdmin().disableTable(hdt.getTableName());
HTU.deleteTable(hdt.getTableName());
}

View File

@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
@ -65,7 +64,7 @@ public class TestScanWithoutFetchingData {
private static final int COUNT = 10;
private static HRegionInfo HRI;
private static RegionInfo HRI;
private static ClientProtos.ClientService.BlockingInterface STUB;
@ -77,7 +76,7 @@ public class TestScanWithoutFetchingData {
table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
}
}
HRI = UTIL.getAdmin().getTableRegions(TABLE_NAME).get(0);
HRI = UTIL.getAdmin().getRegions(TABLE_NAME).get(0);
STUB = ((ConnectionImplementation) UTIL.getConnection())
.getClient(UTIL.getHBaseCluster().getRegionServer(0).getServerName());
}

View File

@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@ -80,7 +79,7 @@ public class TestSnapshotCloneIndependence {
private TableName cloneTableName;
private int countOriginalTable;
String snapshotNameAsString;
byte[] snapshotName;
String snapshotName;
/**
* Setup the config for the cluster and start it
@ -127,7 +126,7 @@ public class TestSnapshotCloneIndependence {
originalTableName = TableName.valueOf("test" + testName.getMethodName());
cloneTableName = TableName.valueOf("test-clone-" + originalTableName);
snapshotNameAsString = "snapshot_" + originalTableName;
snapshotName = Bytes.toBytes(snapshotNameAsString);
snapshotName = snapshotNameAsString;
originalTable = createTable(originalTableName, TEST_FAM);
loadData(originalTable, TEST_FAM);
@ -297,21 +296,21 @@ public class TestSnapshotCloneIndependence {
*/
private void runTestRegionOperationsIndependent() throws Exception {
// Verify that region information is the same pre-split
((ClusterConnection) UTIL.getConnection()).clearRegionCache();
List<HRegionInfo> originalTableHRegions = admin.getTableRegions(originalTableName);
UTIL.getConnection().clearRegionLocationCache();
List<RegionInfo> originalTableHRegions = admin.getRegions(originalTableName);
final int originalRegionCount = originalTableHRegions.size();
final int cloneTableRegionCount = admin.getTableRegions(cloneTableName).size();
final int cloneTableRegionCount = admin.getRegions(cloneTableName).size();
Assert.assertEquals(
"The number of regions in the cloned table is different than in the original table.",
originalRegionCount, cloneTableRegionCount);
// Split a region on the parent table
admin.splitRegion(originalTableHRegions.get(0).getRegionName());
admin.splitRegionAsync(originalTableHRegions.get(0).getRegionName(), null).get();
waitOnSplit(UTIL.getConnection(), originalTable, originalRegionCount);
// Verify that the cloned table region is not split
final int cloneTableRegionCount2 = admin.getTableRegions(cloneTableName).size();
final int cloneTableRegionCount2 = admin.getRegions(cloneTableName).size();
Assert.assertEquals(
"The number of regions in the cloned table changed though none of its regions were split.",
cloneTableRegionCount, cloneTableRegionCount2);
@ -335,8 +334,9 @@ public class TestSnapshotCloneIndependence {
// get a description of the cloned table
// get a list of its families
// assert that the family is there
HTableDescriptor originalTableDescriptor = originalTable.getTableDescriptor();
HTableDescriptor clonedTableDescriptor = admin.getTableDescriptor(cloneTableName);
HTableDescriptor originalTableDescriptor = new HTableDescriptor(originalTable.getDescriptor());
HTableDescriptor clonedTableDescriptor =
new HTableDescriptor(admin.getDescriptor(cloneTableName));
Assert.assertTrue("The original family was not found. There is something wrong. ",
originalTableDescriptor.hasFamily(TEST_FAM));

View File

@ -146,19 +146,10 @@ public class TestSnapshotFromClient {
/**
* Test snapshotting not allowed hbase:meta and -ROOT-
* @throws Exception
*/
@Test
@Test(expected = IllegalArgumentException.class)
public void testMetaTablesSnapshot() throws Exception {
Admin admin = UTIL.getAdmin();
byte[] snapshotName = Bytes.toBytes("metaSnapshot");
try {
admin.snapshot(snapshotName, TableName.META_TABLE_NAME);
fail("taking a snapshot of hbase:meta should not be allowed");
} catch (IllegalArgumentException e) {
// expected
}
UTIL.getAdmin().snapshot("metaSnapshot", TableName.META_TABLE_NAME);
}
/**
@ -177,16 +168,16 @@ public class TestSnapshotFromClient {
UTIL.loadTable(table, TEST_FAM);
table.close();
byte[] snapshot1 = Bytes.toBytes("TableSnapshot1");
String snapshot1 = "TableSnapshot1";
admin.snapshot(snapshot1, TABLE_NAME);
LOG.debug("Snapshot1 completed.");
byte[] snapshot2 = Bytes.toBytes("TableSnapshot2");
String snapshot2 = "TableSnapshot2";
admin.snapshot(snapshot2, TABLE_NAME);
LOG.debug("Snapshot2 completed.");
String snapshot3 = "3rdTableSnapshot";
admin.snapshot(Bytes.toBytes(snapshot3), TABLE_NAME);
admin.snapshot(snapshot3, TABLE_NAME);
LOG.debug(snapshot3 + " completed.");
// delete the first two snapshots
@ -260,13 +251,13 @@ public class TestSnapshotFromClient {
// make sure the table doesn't exist
boolean fail = false;
do {
try {
admin.getTableDescriptor(TableName.valueOf(tableName));
fail = true;
LOG.error("Table:" + tableName + " already exists, checking a new name");
tableName = tableName+"!";
} catch (TableNotFoundException e) {
fail = false;
try {
admin.getDescriptor(TableName.valueOf(tableName));
fail = true;
LOG.error("Table:" + tableName + " already exists, checking a new name");
tableName = tableName + "!";
} catch (TableNotFoundException e) {
fail = false;
}
} while (fail);
@ -297,7 +288,7 @@ public class TestSnapshotFromClient {
FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
// take a snapshot of the disabled table
byte[] snapshot = Bytes.toBytes("testOfflineTableSnapshotWithEmptyRegions");
String snapshot = "testOfflineTableSnapshotWithEmptyRegions";
admin.snapshot(snapshot, TABLE_NAME);
LOG.debug("Snapshot completed.");
@ -342,7 +333,7 @@ public class TestSnapshotFromClient {
LOG.debug("Snapshot2 completed.");
String table2Snapshot1 = "Table2Snapshot1";
admin.snapshot(Bytes.toBytes(table2Snapshot1), tableName);
admin.snapshot(table2Snapshot1, tableName);
LOG.debug(table2Snapshot1 + " completed.");
List<SnapshotDescription> listTableSnapshots =
@ -384,7 +375,7 @@ public class TestSnapshotFromClient {
LOG.debug("Snapshot2 completed.");
String table2Snapshot1 = "Table2Snapshot1";
admin.snapshot(Bytes.toBytes(table2Snapshot1), TABLE_NAME);
admin.snapshot(table2Snapshot1, TABLE_NAME);
LOG.debug(table2Snapshot1 + " completed.");
List<SnapshotDescription> listTableSnapshots =
@ -427,7 +418,7 @@ public class TestSnapshotFromClient {
LOG.debug("Snapshot2 completed.");
String table2Snapshot1 = "Table2Snapshot1";
admin.snapshot(Bytes.toBytes(table2Snapshot1), tableName);
admin.snapshot(table2Snapshot1, tableName);
LOG.debug(table2Snapshot1 + " completed.");
Pattern tableNamePattern = Pattern.compile("test.*");
@ -459,7 +450,7 @@ public class TestSnapshotFromClient {
LOG.debug("Snapshot2 completed.");
String table2Snapshot1 = "Table2Snapshot1";
admin.snapshot(Bytes.toBytes(table2Snapshot1), TABLE_NAME);
admin.snapshot(table2Snapshot1, TABLE_NAME);
LOG.debug(table2Snapshot1 + " completed.");
admin.deleteTableSnapshots(tableNamePattern, Pattern.compile("Table1.*"));

View File

@ -178,7 +178,7 @@ public class TestSnapshotMetadata {
admin.createTable(htd);
Table original = UTIL.getConnection().getTable(originalTableName);
originalTableName = TableName.valueOf(sourceTableNameAsString);
originalTableDescriptor = admin.getTableDescriptor(originalTableName);
originalTableDescriptor = new HTableDescriptor(admin.getDescriptor(originalTableName));
originalTableDescription = originalTableDescriptor.toStringCustomizedValues();
original.close();
@ -195,7 +195,7 @@ public class TestSnapshotMetadata {
final TableName clonedTableName = TableName.valueOf(clonedTableNameAsString);
final String snapshotNameAsString = "snapshot" + originalTableName
+ System.currentTimeMillis();
final byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
final String snapshotName = snapshotNameAsString;
// restore the snapshot into a cloned table and examine the output
List<byte[]> familiesList = new ArrayList<>();
@ -207,7 +207,7 @@ public class TestSnapshotMetadata {
admin.cloneSnapshot(snapshotName, clonedTableName);
Table clonedTable = UTIL.getConnection().getTable(clonedTableName);
HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName);
HTableDescriptor cloneHtd = new HTableDescriptor(admin.getDescriptor(clonedTableName));
assertEquals(
originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString),
cloneHtd.toStringCustomizedValues());
@ -287,7 +287,6 @@ public class TestSnapshotMetadata {
// take a "disabled" snapshot
final String snapshotNameAsString = "snapshot" + originalTableName
+ System.currentTimeMillis();
final byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
SnapshotTestingUtils.createSnapshotAndValidate(admin, originalTableName,
familiesWithDataList, emptyFamiliesList, snapshotNameAsString, rootDir, fs,
@ -303,7 +302,7 @@ public class TestSnapshotMetadata {
HColumnDescriptor hcd = new HColumnDescriptor(newFamilyName);
admin.addColumnFamily(originalTableName, hcd);
assertTrue("New column family was not added.",
admin.getTableDescriptor(originalTableName).toString().contains(newFamilyNameAsString));
admin.getDescriptor(originalTableName).toString().contains(newFamilyNameAsString));
}
// restore it
@ -311,16 +310,14 @@ public class TestSnapshotMetadata {
admin.disableTable(originalTableName);
}
admin.restoreSnapshot(snapshotName);
admin.restoreSnapshot(snapshotNameAsString);
admin.enableTable(originalTableName);
// verify that the descrption is reverted
Table original = UTIL.getConnection().getTable(originalTableName);
try {
assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName)));
assertTrue(originalTableDescriptor.equals(original.getTableDescriptor()));
} finally {
original.close();
try (Table original = UTIL.getConnection().getTable(originalTableName)) {
assertEquals(originalTableDescriptor,
new HTableDescriptor(admin.getDescriptor(originalTableName)));
assertEquals(originalTableDescriptor, new HTableDescriptor(original.getDescriptor()));
}
}
}

View File

@ -29,6 +29,7 @@ import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.UUID;
import java.util.regex.Pattern;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -152,9 +153,9 @@ public class TestSnapshotTemporaryDirectory {
throws IOException, InterruptedException {
long tid = System.currentTimeMillis();
TableName tableName = TableName.valueOf("testtb-" + tid);
byte[] emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
byte[] snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
byte[] snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
String emptySnapshot = "emptySnaptb-" + tid;
String snapshotName0 = "snaptb0-" + tid;
String snapshotName1 = "snaptb1-" + tid;
int snapshot0Rows;
int snapshot1Rows;
@ -163,7 +164,7 @@ public class TestSnapshotTemporaryDirectory {
admin.disableTable(tableName);
// take an empty snapshot
takeSnapshot(tableName, Bytes.toString(emptySnapshot), true);
takeSnapshot(tableName, emptySnapshot, true);
// enable table and insert data
admin.enableTable(tableName);
@ -174,7 +175,7 @@ public class TestSnapshotTemporaryDirectory {
admin.disableTable(tableName);
// take a snapshot
takeSnapshot(tableName, Bytes.toString(snapshotName0), true);
takeSnapshot(tableName, snapshotName0, true);
// enable table and insert more data
admin.enableTable(tableName);
@ -185,7 +186,7 @@ public class TestSnapshotTemporaryDirectory {
SnapshotTestingUtils.verifyRowCount(UTIL, tableName, snapshot1Rows);
admin.disableTable(tableName);
takeSnapshot(tableName, Bytes.toString(snapshotName1), true);
takeSnapshot(tableName, snapshotName1, true);
// Restore from snapshot-0
admin.restoreSnapshot(snapshotName0);
@ -218,9 +219,9 @@ public class TestSnapshotTemporaryDirectory {
throws IOException, InterruptedException {
long tid = System.currentTimeMillis();
TableName tableName = TableName.valueOf("testtb-" + tid);
byte[] emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
byte[] snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
byte[] snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
String emptySnapshot = "emptySnaptb-" + tid;
String snapshotName0 = "snaptb0-" + tid;
String snapshotName1 = "snaptb1-" + tid;
int snapshot0Rows;
int snapshot1Rows;
@ -228,7 +229,7 @@ public class TestSnapshotTemporaryDirectory {
SnapshotTestingUtils.createTable(UTIL, tableName, getNumReplicas(), TEST_FAM);
// take an empty snapshot
takeSnapshot(tableName, Bytes.toString(emptySnapshot), false);
takeSnapshot(tableName, emptySnapshot, false);
// Insert data
SnapshotTestingUtils.loadData(UTIL, tableName, 500, TEST_FAM);
@ -237,7 +238,7 @@ public class TestSnapshotTemporaryDirectory {
}
// take a snapshot
takeSnapshot(tableName, Bytes.toString(snapshotName0), false);
takeSnapshot(tableName, snapshotName0, false);
// Insert more data
SnapshotTestingUtils.loadData(UTIL, tableName, 500, TEST_FAM);
@ -246,7 +247,7 @@ public class TestSnapshotTemporaryDirectory {
}
SnapshotTestingUtils.verifyRowCount(UTIL, tableName, snapshot1Rows);
takeSnapshot(tableName, Bytes.toString(snapshotName1), false);
takeSnapshot(tableName, snapshotName1, false);
// Restore from snapshot-0
admin.disableTable(tableName);
@ -416,12 +417,11 @@ public class TestSnapshotTemporaryDirectory {
}
// Ensures that the snapshot is transferred to the proper completed snapshot directory
@Test(timeout = 180000) public void testEnsureTemporaryDirectoryTransfer() throws Exception {
Admin admin = null;
@Test(timeout = 180000)
public void testEnsureTemporaryDirectoryTransfer() throws Exception {
Admin admin = UTIL.getAdmin();
TableName tableName2 = TableName.valueOf("testListTableSnapshots");
try {
admin = UTIL.getHBaseAdmin();
HTableDescriptor htd = new HTableDescriptor(tableName2);
UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
@ -437,7 +437,8 @@ public class TestSnapshotTemporaryDirectory {
takeSnapshot(TABLE_NAME, table2Snapshot1, false);
LOG.debug("Table2Snapshot1 completed.");
List<SnapshotDescription> listTableSnapshots = admin.listTableSnapshots("test.*", ".*");
List<SnapshotDescription> listTableSnapshots =
admin.listTableSnapshots(Pattern.compile("test.*"), Pattern.compile(".*"));
List<String> listTableSnapshotNames = new ArrayList<String>();
assertEquals(3, listTableSnapshots.size());
for (SnapshotDescription s : listTableSnapshots) {
@ -447,16 +448,14 @@ public class TestSnapshotTemporaryDirectory {
assertTrue(listTableSnapshotNames.contains(table1Snapshot2));
assertTrue(listTableSnapshotNames.contains(table2Snapshot1));
} finally {
if (admin != null) {
try {
admin.deleteSnapshots("Table.*");
} catch (SnapshotDoesNotExistException ignore) {
}
if (admin.tableExists(tableName2)) {
UTIL.deleteTable(tableName2);
}
admin.close();
try {
admin.deleteSnapshots(Pattern.compile("Table.*"));
} catch (SnapshotDoesNotExistException ignore) {
}
if (admin.tableExists(tableName2)) {
UTIL.deleteTable(tableName2);
}
admin.close();
}
}

View File

@ -17,7 +17,6 @@
*/
package org.apache.hadoop.hbase.client;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -29,7 +28,6 @@ import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -84,20 +82,18 @@ public class TestSplitOrMergeStatus {
Admin admin = TEST_UTIL.getAdmin();
initSwitchStatus(admin);
boolean[] results = admin.setSplitOrMergeEnabled(false, false, MasterSwitchType.SPLIT);
assertEquals(1, results.length);
assertTrue(results[0]);
boolean result = admin.splitSwitch(false, false);
assertTrue(result);
admin.split(t.getName());
int count = admin.getTableRegions(tableName).size();
int count = admin.getRegions(tableName).size();
assertTrue(originalCount == count);
results = admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.SPLIT);
assertEquals(1, results.length);
assertFalse(results[0]);
result = admin.splitSwitch(true, false);
assertFalse(result);
admin.split(t.getName());
while ((count = admin.getTableRegions(tableName).size()) == originalCount) {
while ((count = admin.getRegions(tableName).size()) == originalCount) {
Threads.sleep(1);
}
count = admin.getTableRegions(tableName).size();
count = admin.getRegions(tableName).size();
assertTrue(originalCount < count);
admin.close();
}
@ -110,21 +106,20 @@ public class TestSplitOrMergeStatus {
TEST_UTIL.loadTable(t, FAMILY, false);
Admin admin = TEST_UTIL.getAdmin();
int originalCount = admin.getTableRegions(tableName).size();
int originalCount = admin.getRegions(tableName).size();
initSwitchStatus(admin);
admin.split(t.getName());
int postSplitCount = -1;
while ((postSplitCount = admin.getTableRegions(tableName).size()) == originalCount) {
while ((postSplitCount = admin.getRegions(tableName).size()) == originalCount) {
Threads.sleep(1);
}
assertTrue("originalCount=" + originalCount + ", newCount=" + postSplitCount,
originalCount != postSplitCount);
// Merge switch is off so merge should NOT succeed.
boolean[] results = admin.setSplitOrMergeEnabled(false, false, MasterSwitchType.MERGE);
assertEquals(1, results.length);
assertTrue(results[0]);
List<HRegionInfo> regions = admin.getTableRegions(t.getName());
boolean result = admin.mergeSwitch(false, false);
assertTrue(result);
List<RegionInfo> regions = admin.getRegions(t.getName());
assertTrue(regions.size() > 1);
Future<?> f = admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
regions.get(1).getEncodedNameAsBytes(), true);
@ -134,17 +129,16 @@ public class TestSplitOrMergeStatus {
} catch (ExecutionException ee) {
// Expected.
}
int count = admin.getTableRegions(tableName).size();
int count = admin.getRegions(tableName).size();
assertTrue("newCount=" + postSplitCount + ", count=" + count, postSplitCount == count);
results = admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.MERGE);
regions = admin.getTableRegions(t.getName());
assertEquals(1, results.length);
assertFalse(results[0]);
result = admin.mergeSwitch(true, false);
regions = admin.getRegions(t.getName());
assertFalse(result);
f = admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
regions.get(1).getEncodedNameAsBytes(), true);
f.get(10, TimeUnit.SECONDS);
count = admin.getTableRegions(tableName).size();
count = admin.getRegions(tableName).size();
assertTrue((postSplitCount / 2 /*Merge*/) == count);
admin.close();
}
@ -152,24 +146,22 @@ public class TestSplitOrMergeStatus {
@Test
public void testMultiSwitches() throws IOException {
Admin admin = TEST_UTIL.getAdmin();
boolean[] switches = admin.setSplitOrMergeEnabled(false, false,
MasterSwitchType.SPLIT, MasterSwitchType.MERGE);
for (boolean s : switches){
assertTrue(s);
}
assertFalse(admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT));
assertFalse(admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE));
assertTrue(admin.splitSwitch(false, false));
assertTrue(admin.mergeSwitch(false, false));
assertFalse(admin.isSplitEnabled());
assertFalse(admin.isMergeEnabled());
admin.close();
}
private void initSwitchStatus(Admin admin) throws IOException {
if (!admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {
admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.SPLIT);
if (!admin.isSplitEnabled()) {
admin.splitSwitch(true, false);
}
if (!admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {
admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.MERGE);
if (!admin.isMergeEnabled()) {
admin.mergeSwitch(true, false);
}
assertTrue(admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT));
assertTrue(admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE));
assertTrue(admin.isSplitEnabled());
assertTrue(admin.isMergeEnabled());
}
}

View File

@ -114,8 +114,8 @@ public class TestTableFavoredNodes {
public void setup() throws IOException {
fnm = TEST_UTIL.getMiniHBaseCluster().getMaster().getFavoredNodesManager();
admin = TEST_UTIL.getAdmin();
admin.setBalancerRunning(false, true);
admin.enableCatalogJanitor(false);
admin.balancerSwitch(false, true);
admin.catalogJanitorSwitch(false);
}
/*
@ -130,7 +130,7 @@ public class TestTableFavoredNodes {
// All regions should have favored nodes
checkIfFavoredNodeInformationIsCorrect(tableName);
List<HRegionInfo> regions = admin.getTableRegions(tableName);
List<RegionInfo> regions = admin.getRegions(tableName);
TEST_UTIL.deleteTable(tableName);
@ -149,13 +149,13 @@ public class TestTableFavoredNodes {
// All regions should have favored nodes
checkIfFavoredNodeInformationIsCorrect(tableName);
List<HRegionInfo> regions = admin.getTableRegions(tableName);
List<RegionInfo> regions = admin.getRegions(tableName);
TEST_UTIL.truncateTable(tableName, true);
checkNoFNForDeletedTable(regions);
checkIfFavoredNodeInformationIsCorrect(tableName);
regions = admin.getTableRegions(tableName);
regions = admin.getRegions(tableName);
TEST_UTIL.truncateTable(tableName, false);
checkNoFNForDeletedTable(regions);
@ -170,7 +170,7 @@ public class TestTableFavoredNodes {
final TableName tableName = TableName.valueOf(name.getMethodName());
Table t = TEST_UTIL.createTable(tableName, Bytes.toBytes("f"), splitKeys);
TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
final int numberOfRegions = admin.getTableRegions(t.getName()).size();
final int numberOfRegions = admin.getRegions(t.getName()).size();
checkIfFavoredNodeInformationIsCorrect(tableName);
@ -211,14 +211,14 @@ public class TestTableFavoredNodes {
// Major compact table and run catalog janitor. Parent's FN should be removed
TEST_UTIL.getMiniHBaseCluster().compact(tableName, true);
admin.runCatalogScan();
admin.runCatalogJanitor();
// Catalog cleanup is async. Wait on procedure to finish up.
ProcedureTestingUtility.waitAllProcedures(
TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor());
// assertEquals("Parent region should have been cleaned", 1, admin.runCatalogScan());
assertNull("Parent FN should be null", fnm.getFavoredNodes(parent));
List<HRegionInfo> regions = admin.getTableRegions(tableName);
List<RegionInfo> regions = admin.getRegions(tableName);
// Split and Table Disable interfere with each other around region replicas
// TODO. Meantime pause a few seconds.
Threads.sleep(2000);
@ -266,22 +266,22 @@ public class TestTableFavoredNodes {
// Major compact table and run catalog janitor. Parent FN should be removed
TEST_UTIL.getMiniHBaseCluster().compact(tableName, true);
assertEquals("Merge parents should have been cleaned", 1, admin.runCatalogScan());
assertEquals("Merge parents should have been cleaned", 1, admin.runCatalogJanitor());
// Catalog cleanup is async. Wait on procedure to finish up.
ProcedureTestingUtility.waitAllProcedures(
TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor());
assertNull("Parent FN should be null", fnm.getFavoredNodes(regionA));
assertNull("Parent FN should be null", fnm.getFavoredNodes(regionB));
List<HRegionInfo> regions = admin.getTableRegions(tableName);
List<RegionInfo> regions = admin.getRegions(tableName);
TEST_UTIL.deleteTable(tableName);
checkNoFNForDeletedTable(regions);
}
private void checkNoFNForDeletedTable(List<HRegionInfo> regions) {
for (HRegionInfo region : regions) {
private void checkNoFNForDeletedTable(List<RegionInfo> regions) {
for (RegionInfo region : regions) {
LOG.info("Testing if FN data for " + region);
assertNull("FN not null for deleted table's region: " + region, fnm.getFavoredNodes(region));
}
@ -362,8 +362,8 @@ public class TestTableFavoredNodes {
for (TableName sysTable :
admin.listTableNamesByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
List<HRegionInfo> regions = admin.getTableRegions(sysTable);
for (HRegionInfo region : regions) {
List<RegionInfo> regions = admin.getRegions(sysTable);
for (RegionInfo region : regions) {
assertNull("FN should be null for sys region", fnm.getFavoredNodes(region));
}
}

View File

@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.StartMiniClusterOption;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@ -65,7 +64,7 @@ public class TestTableSnapshotScanner {
public static void blockUntilSplitFinished(HBaseTestingUtility util, TableName tableName,
int expectedRegionSize) throws Exception {
for (int i = 0; i < 100; i++) {
List<HRegionInfo> hRegionInfoList = util.getAdmin().getTableRegions(tableName);
List<RegionInfo> hRegionInfoList = util.getAdmin().getRegions(tableName);
if (hRegionInfoList.size() >= expectedRegionSize) {
break;
}

View File

@ -93,15 +93,15 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
@Test
public void disableNotFullReplication() throws Exception {
HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
HTableDescriptor table = new HTableDescriptor(admin2.getDescriptor(tableName));
HColumnDescriptor f = new HColumnDescriptor("notReplicatedFamily");
table.addFamily(f);
admin1.disableTable(tableName);
admin1.modifyTable(tableName, table);
admin1.modifyTable(table);
admin1.enableTable(tableName);
admin1.disableTableReplication(tableName);
table = admin1.getTableDescriptor(tableName);
table = new HTableDescriptor(admin1.getDescriptor(tableName));
for (HColumnDescriptor fam : table.getColumnFamilies()) {
assertEquals(HConstants.REPLICATION_SCOPE_LOCAL, fam.getScope());
}
@ -121,20 +121,20 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
@Test
public void testEnableReplicationWhenReplicationNotEnabled() throws Exception {
HTableDescriptor table = new HTableDescriptor(admin1.getTableDescriptor(tableName));
HTableDescriptor table = new HTableDescriptor(admin1.getDescriptor(tableName));
for (HColumnDescriptor fam : table.getColumnFamilies()) {
fam.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
}
admin1.disableTable(tableName);
admin1.modifyTable(tableName, table);
admin1.modifyTable(table);
admin1.enableTable(tableName);
admin2.disableTable(tableName);
admin2.modifyTable(tableName, table);
admin2.modifyTable(table);
admin2.enableTable(tableName);
admin1.enableTableReplication(tableName);
table = admin1.getTableDescriptor(tableName);
table = new HTableDescriptor(admin1.getDescriptor(tableName));
for (HColumnDescriptor fam : table.getColumnFamilies()) {
assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
}
@ -142,11 +142,11 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
@Test
public void testEnableReplicationWhenTableDescriptorIsNotSameInClusters() throws Exception {
HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
HTableDescriptor table = new HTableDescriptor(admin2.getDescriptor(tableName));
HColumnDescriptor f = new HColumnDescriptor("newFamily");
table.addFamily(f);
admin2.disableTable(tableName);
admin2.modifyTable(tableName, table);
admin2.modifyTable(table);
admin2.enableTable(tableName);
try {
@ -156,10 +156,10 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
}
admin1.disableTable(tableName);
admin1.modifyTable(tableName, table);
admin1.modifyTable(table);
admin1.enableTable(tableName);
admin1.enableTableReplication(tableName);
table = admin1.getTableDescriptor(tableName);
table = new HTableDescriptor(admin1.getDescriptor(tableName));
for (HColumnDescriptor fam : table.getColumnFamilies()) {
assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
}
@ -171,12 +171,12 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
@Test
public void testDisableAndEnableReplication() throws Exception {
admin1.disableTableReplication(tableName);
HTableDescriptor table = admin1.getTableDescriptor(tableName);
HTableDescriptor table = new HTableDescriptor(admin1.getDescriptor(tableName));
for (HColumnDescriptor fam : table.getColumnFamilies()) {
assertEquals(HConstants.REPLICATION_SCOPE_LOCAL, fam.getScope());
}
admin1.enableTableReplication(tableName);
table = admin1.getTableDescriptor(tableName);
table = new HTableDescriptor(admin1.getDescriptor(tableName));
for (HColumnDescriptor fam : table.getColumnFamilies()) {
assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
}

View File

@ -286,7 +286,7 @@ public class TestCoprocessorMetrics {
public void setup() throws IOException {
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
Admin admin = connection.getAdmin()) {
for (HTableDescriptor htd : admin.listTables()) {
for (TableDescriptor htd : admin.listTableDescriptors()) {
UTIL.deleteTable(htd.getTableName());
}
}

View File

@ -1256,11 +1256,11 @@ public class TestMasterObserver {
}
private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static byte[] TEST_SNAPSHOT = Bytes.toBytes("observed_snapshot");
private static String TEST_SNAPSHOT = "observed_snapshot";
private static TableName TEST_CLONE = TableName.valueOf("observed_clone");
private static byte[] TEST_FAMILY = Bytes.toBytes("fam1");
private static byte[] TEST_FAMILY2 = Bytes.toBytes("fam2");
@Rule public TestName name = new TestName();
@Rule
public TestName name = new TestName();
@BeforeClass
public static void setupBeforeClass() throws Exception {
@ -1507,10 +1507,10 @@ public class TestMasterObserver {
private void modifyTableSync(Admin admin, TableName tableName, HTableDescriptor htd)
throws IOException {
admin.modifyTable(tableName, htd);
admin.modifyTable(htd);
//wait until modify table finishes
for (int t = 0; t < 100; t++) { //10 sec timeout
HTableDescriptor td = admin.getTableDescriptor(htd.getTableName());
HTableDescriptor td = new HTableDescriptor(admin.getDescriptor(htd.getTableName()));
if (td.equals(htd)) {
break;
}

View File

@ -22,9 +22,9 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@ -80,7 +80,7 @@ public class TestLoadAndSwitchEncodeOnDisk extends
compression = Compression.Algorithm.GZ; // used for table setup
super.loadTest();
HColumnDescriptor hcd = getColumnDesc(admin);
ColumnFamilyDescriptor hcd = getColumnDesc(admin);
System.err.println("\nDisabling encode-on-disk. Old column descriptor: " + hcd + "\n");
Table t = TEST_UTIL.getConnection().getTable(TABLE);
assertAllOnLine(t);

View File

@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.ExecutionException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -91,13 +92,14 @@ public class TestCatalogJanitorInMemoryStates {
* Test clearing a split parent from memory.
*/
@Test
public void testInMemoryParentCleanup() throws IOException, InterruptedException {
public void testInMemoryParentCleanup()
throws IOException, InterruptedException, ExecutionException {
final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
final ServerManager sm = TEST_UTIL.getHBaseCluster().getMaster().getServerManager();
final CatalogJanitor janitor = TEST_UTIL.getHBaseCluster().getMaster().getCatalogJanitor();
Admin admin = TEST_UTIL.getAdmin();
admin.enableCatalogJanitor(false);
admin.catalogJanitorSwitch(false);
final TableName tableName = TableName.valueOf(name.getMethodName());
Table t = TEST_UTIL.createTable(tableName, FAMILY);
@ -129,19 +131,18 @@ public class TestCatalogJanitorInMemoryStates {
}
/*
* Splits a region
* @param t Region to split.
* @return List of region locations
* @throws IOException, InterruptedException
*/
/**
* Splits a region
* @param t Region to split.
* @return List of region locations
*/
private List<HRegionLocation> splitRegion(final RegionInfo r)
throws IOException, InterruptedException {
throws IOException, InterruptedException, ExecutionException {
List<HRegionLocation> locations = new ArrayList<>();
// Split this table in two.
Admin admin = TEST_UTIL.getAdmin();
Connection connection = TEST_UTIL.getConnection();
admin.splitRegion(r.getEncodedNameAsBytes());
admin.splitRegionAsync(r.getEncodedNameAsBytes(), null).get();
admin.close();
PairOfSameType<RegionInfo> regions = waitOnDaughters(r);
if (regions != null) {

View File

@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -147,7 +147,7 @@ public class TestMasterBalanceThrottling {
Thread.sleep(100);
}
HRegionServer biasedServer = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
for (HRegionInfo regionInfo : TEST_UTIL.getAdmin().getTableRegions(tableName)) {
for (RegionInfo regionInfo : TEST_UTIL.getAdmin().getRegions(tableName)) {
master.move(regionInfo.getEncodedNameAsBytes(),
Bytes.toBytes(biasedServer.getServerName().getServerName()));
}

View File

@ -219,9 +219,9 @@ public class TestMasterOperationsForRegionReplicas {
assert(ADMIN.isTableDisabled(tableName));
//increase the replica
desc.setRegionReplication(numReplica + 1);
ADMIN.modifyTable(tableName, desc);
ADMIN.modifyTable(desc);
ADMIN.enableTable(tableName);
LOG.info(ADMIN.getTableDescriptor(tableName).toString());
LOG.info(ADMIN.getDescriptor(tableName).toString());
assert(ADMIN.isTableEnabled(tableName));
List<RegionInfo> regions = TEST_UTIL.getMiniHBaseCluster().getMaster().
getAssignmentManager().getRegionStates().getRegionsOfTable(tableName);
@ -231,7 +231,7 @@ public class TestMasterOperationsForRegionReplicas {
//decrease the replica(earlier, table was modified to have a replica count of numReplica + 1)
ADMIN.disableTable(tableName);
desc.setRegionReplication(numReplica);
ADMIN.modifyTable(tableName, desc);
ADMIN.modifyTable(desc);
ADMIN.enableTable(tableName);
assert(ADMIN.isTableEnabled(tableName));
regions = TEST_UTIL.getMiniHBaseCluster().getMaster()

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.io.StringWriter;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
@ -118,11 +120,9 @@ public class TestMasterStatusServlet {
}
private void setupMockTables() throws IOException {
HTableDescriptor tables[] = new HTableDescriptor[] {
new HTableDescriptor(TableName.valueOf("foo")),
new HTableDescriptor(TableName.valueOf("bar"))
};
Mockito.doReturn(tables).when(admin).listTables();
List<TableDescriptor> tables = Arrays.asList(new HTableDescriptor(TableName.valueOf("foo")),
new HTableDescriptor(TableName.valueOf("bar")));
Mockito.doReturn(tables).when(admin).listTableDescriptors();
}
@Test

View File

@ -106,7 +106,7 @@ public class TestAssignmentOnRSCrash {
throws Exception {
final int NROWS = 100;
int nkilled = 0;
for (RegionInfo hri: UTIL.getHBaseAdmin().getTableRegions(TEST_TABLE)) {
for (RegionInfo hri: UTIL.getHBaseAdmin().getRegions(TEST_TABLE)) {
ServerName serverName = AssignmentTestingUtil.getServerHoldingRegion(UTIL, hri);
if (AssignmentTestingUtil.isServerHoldingMeta(UTIL, serverName)) continue;

View File

@ -26,13 +26,13 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.HMaster;
@ -125,7 +125,7 @@ public class TestRogueRSAssignment {
@Before
public void setup() throws IOException {
// Turn off balancer
admin.setBalancerRunning(false, true);
admin.balancerSwitch(false, true);
}
@After
@ -135,7 +135,7 @@ public class TestRogueRSAssignment {
UTIL.deleteTable(td.getTableName());
}
// Turn on balancer
admin.setBalancerRunning(true, false);
admin.balancerSwitch(true, false);
}
/**
@ -146,7 +146,7 @@ public class TestRogueRSAssignment {
public void testReportRSWithWrongRegion() throws Exception {
final TableName tableName = TableName.valueOf(this.name.getMethodName());
List<HRegionInfo> tableRegions = createTable(tableName);
List<RegionInfo> tableRegions = createTable(tableName);
final ServerName sn = ServerName.parseVersionedServerName(
ServerName.valueOf("1.example.org", 1, System.currentTimeMillis()).getVersionedBytes());
@ -164,7 +164,7 @@ public class TestRogueRSAssignment {
}
private RegionServerStatusProtos.RegionServerReportRequest.Builder
makeRSReportRequestWithRegions(final ServerName sn, HRegionInfo... regions) {
makeRSReportRequestWithRegions(final ServerName sn, RegionInfo... regions) {
ClusterStatusProtos.ServerLoad.Builder sl = ClusterStatusProtos.ServerLoad.newBuilder();
for (int i = 0; i < regions.length; i++) {
HBaseProtos.RegionSpecifier.Builder rs = HBaseProtos.RegionSpecifier.newBuilder();
@ -182,7 +182,7 @@ public class TestRogueRSAssignment {
.setLoad(sl);
}
private List<HRegionInfo> createTable(final TableName tableName) throws Exception {
private List<RegionInfo> createTable(final TableName tableName) throws Exception {
TableDescriptorBuilder tdBuilder = TableDescriptorBuilder.newBuilder(tableName);
tdBuilder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build());
@ -194,10 +194,10 @@ public class TestRogueRSAssignment {
return assertRegionCount(tableName, initialRegionCount);
}
private List<HRegionInfo> assertRegionCount(final TableName tableName, final int nregions)
private List<RegionInfo> assertRegionCount(final TableName tableName, final int nregions)
throws Exception {
UTIL.waitUntilNoRegionsInTransition();
List<HRegionInfo> tableRegions = admin.getTableRegions(tableName);
List<RegionInfo> tableRegions = admin.getRegions(tableName);
assertEquals(nregions, tableRegions.size());
return tableRegions;
}

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master.balancer;
import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORED_NODES_NUM;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import java.util.List;
@ -29,11 +28,11 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -82,7 +81,7 @@ public class TestFavoredNodeTableImport {
Threads.sleep(1);
}
Admin admin = UTIL.getAdmin();
admin.setBalancerRunning(false, true);
admin.balancerSwitch(false, true);
String tableName = "testFNImport";
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
@ -107,8 +106,8 @@ public class TestFavoredNodeTableImport {
FavoredNodesManager fnm = UTIL.getHBaseCluster().getMaster().getFavoredNodesManager();
List<HRegionInfo> regionsOfTable = admin.getTableRegions(TableName.valueOf(tableName));
for (HRegionInfo rInfo : regionsOfTable) {
List<RegionInfo> regionsOfTable = admin.getRegions(TableName.valueOf(tableName));
for (RegionInfo rInfo : regionsOfTable) {
Set<ServerName> favNodes = Sets.newHashSet(fnm.getFavoredNodes(rInfo));
assertNotNull(favNodes);
assertEquals("Required no of favored nodes not found.", FAVORED_NODES_NUM, favNodes.size());

View File

@ -109,7 +109,7 @@ public class TestFavoredStochasticBalancerPickers extends BalancerTestBase {
TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(120*1000);
cluster = TEST_UTIL.getMiniHBaseCluster();
admin = TEST_UTIL.getAdmin();
admin.setBalancerRunning(false, true);
admin.balancerSwitch(false, true);
}
@After

View File

@ -104,7 +104,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
cluster = TEST_UTIL.getMiniHBaseCluster();
master = TEST_UTIL.getMiniHBaseCluster().getMaster();
admin = TEST_UTIL.getAdmin();
admin.setBalancerRunning(false, true);
admin.balancerSwitch(false, true);
}
@After
@ -130,8 +130,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
// Now try to run balance, and verify no regions are moved to the 2 region servers recently
// started.
admin.setBalancerRunning(true, true);
assertTrue("Balancer did not run", admin.balancer());
admin.balancerSwitch(true, true);
assertTrue("Balancer did not run", admin.balance());
TEST_UTIL.waitUntilNoRegionsInTransition(120000);
List<RegionInfo> hris = admin.getRegions(rs1.getRegionServer().getServerName());
@ -159,7 +159,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
LoadBalancer balancer = master.getLoadBalancer();
List<RegionInfo> regions = admin.getRegions(tableName);
regions.addAll(admin.getTableRegions(TableName.META_TABLE_NAME));
regions.addAll(admin.getRegions(TableName.META_TABLE_NAME));
List<ServerName> servers = Lists.newArrayList(
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet());
Map<ServerName, List<RegionInfo>> map = balancer.roundRobinAssignment(regions, servers);
@ -219,7 +219,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc);
TEST_UTIL.waitTableAvailable(desc.getTableName());
RegionInfo hri = admin.getTableRegions(TableName.valueOf(tableName)).get(0);
RegionInfo hri = admin.getRegions(TableName.valueOf(tableName)).get(0);
FavoredNodesManager fnm = master.getFavoredNodesManager();
fnm.deleteFavoredNodesForRegions(Lists.newArrayList(hri));
@ -250,7 +250,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
final RegionInfo region = admin.getTableRegions(tableName).get(0);
final RegionInfo region = admin.getRegions(tableName).get(0);
LOG.info("Region thats supposed to be in transition: " + region);
FavoredNodesManager fnm = master.getFavoredNodesManager();
List<ServerName> currentFN = fnm.getFavoredNodes(region);
@ -259,10 +259,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
fnm.deleteFavoredNodesForRegions(Lists.newArrayList(region));
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
admin.setBalancerRunning(true, true);
admin.balancerSwitch(true, true);
// Balancer should unassign the region
assertTrue("Balancer did not run", admin.balancer());
assertTrue("Balancer did not run", admin.balance());
TEST_UTIL.waitUntilNoRegionsInTransition();
admin.assign(region.getEncodedNameAsBytes());
@ -273,7 +273,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
assertEquals("Expected number of FN not present",
FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, currentFN.size());
assertTrue("Balancer did not run", admin.balancer());
assertTrue("Balancer did not run", admin.balance());
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
checkFavoredNodeAssignments(tableName, fnm, regionStates);
@ -288,7 +288,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
final RegionInfo misplacedRegion = admin.getTableRegions(tableName).get(0);
final RegionInfo misplacedRegion = admin.getRegions(tableName).get(0);
FavoredNodesManager fnm = master.getFavoredNodesManager();
List<ServerName> currentFN = fnm.getFavoredNodes(misplacedRegion);
assertNotNull(currentFN);
@ -314,8 +314,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
final ServerName current = regionStates.getRegionServerOfRegion(misplacedRegion);
assertNull("Misplaced region is still hosted on favored node, not expected.",
FavoredNodesPlan.getFavoredServerPosition(fnm.getFavoredNodes(misplacedRegion), current));
admin.setBalancerRunning(true, true);
assertTrue("Balancer did not run", admin.balancer());
admin.balancerSwitch(true, true);
assertTrue("Balancer did not run", admin.balance());
TEST_UTIL.waitFor(120000, 30000, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
@ -335,7 +335,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
final RegionInfo region = admin.getTableRegions(tableName).get(0);
final RegionInfo region = admin.getRegions(tableName).get(0);
LOG.info("Region that's supposed to be in transition: " + region);
FavoredNodesManager fnm = master.getFavoredNodesManager();
List<ServerName> currentFN = fnm.getFavoredNodes(region);
@ -356,9 +356,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
}
});
assertEquals("Not all regions are online", REGION_NUM, admin.getTableRegions(tableName).size());
admin.setBalancerRunning(true, true);
assertTrue("Balancer did not run", admin.balancer());
assertEquals("Not all regions are online", REGION_NUM, admin.getRegions(tableName).size());
admin.balancerSwitch(true, true);
assertTrue("Balancer did not run", admin.balance());
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
checkFavoredNodeAssignments(tableName, fnm, regionStates);
@ -373,7 +373,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
final RegionInfo region = admin.getTableRegions(tableName).get(0);
final RegionInfo region = admin.getRegions(tableName).get(0);
LOG.info("Region that's supposed to be in transition: " + region);
FavoredNodesManager fnm = master.getFavoredNodesManager();
List<ServerName> currentFN = fnm.getFavoredNodes(region);
@ -417,10 +417,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.assign(regionInfo.getEncodedNameAsBytes());
}
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
assertEquals("Not all regions are online", REGION_NUM, admin.getTableRegions(tableName).size());
assertEquals("Not all regions are online", REGION_NUM, admin.getRegions(tableName).size());
admin.setBalancerRunning(true, true);
assertTrue("Balancer did not run", admin.balancer());
admin.balancerSwitch(true, true);
assertTrue("Balancer did not run", admin.balance());
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
checkFavoredNodeAssignments(tableName, fnm, regionStates);
@ -435,7 +435,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
TEST_UTIL.waitTableAvailable(tableName);
final RegionInfo region = admin.getTableRegions(tableName).get(0);
final RegionInfo region = admin.getRegions(tableName).get(0);
LOG.info("Region that's supposed to be in transition: " + region);
FavoredNodesManager fnm = master.getFavoredNodesManager();
List<ServerName> currentFN = fnm.getFavoredNodes(region);
@ -511,10 +511,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
admin.assign(regionInfo.getEncodedNameAsBytes());
}
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
assertEquals("Not all regions are online", REGION_NUM, admin.getTableRegions(tableName).size());
assertEquals("Not all regions are online", REGION_NUM, admin.getRegions(tableName).size());
admin.setBalancerRunning(true, true);
assertTrue("Balancer did not run", admin.balancer());
admin.balancerSwitch(true, true);
assertTrue("Balancer did not run", admin.balance());
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
checkFavoredNodeAssignments(tableName, fnm, regionStates);
@ -522,7 +522,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
private void checkFavoredNodeAssignments(TableName tableName, FavoredNodesManager fnm,
RegionStates regionStates) throws IOException {
for (RegionInfo hri : admin.getTableRegions(tableName)) {
for (RegionInfo hri : admin.getRegions(tableName)) {
ServerName host = regionStates.getRegionServerOfRegion(hri);
assertNotNull("Region: " + hri.getEncodedName() + " not on FN, current: " + host
+ " FN list: " + fnm.getFavoredNodes(hri),

View File

@ -25,9 +25,9 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@ -67,7 +67,7 @@ public class TestLockManager {
private static String namespace = "namespace";
private static TableName tableName = TableName.valueOf(namespace, "table");
private static HRegionInfo[] tableRegions;
private static RegionInfo[] tableRegions;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
@ -83,9 +83,9 @@ public class TestLockManager {
UTIL.getAdmin().createNamespace(NamespaceDescriptor.create(namespace).build());
UTIL.createTable(tableName, new byte[][]{Bytes.toBytes("fam")},
new byte[][] {Bytes.toBytes("1")});
List<HRegionInfo> regions = UTIL.getAdmin().getTableRegions(tableName);
List<RegionInfo> regions = UTIL.getAdmin().getRegions(tableName);
assert regions.size() > 0;
tableRegions = new HRegionInfo[regions.size()];
tableRegions = new RegionInfo[regions.size()];
regions.toArray(tableRegions);
}

View File

@ -141,9 +141,9 @@ public class TestSimpleRegionNormalizerOnCluster {
region.flush(true);
}
HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLENAME));
HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
htd.setNormalizationEnabled(true);
admin.modifyTable(TABLENAME, htd);
admin.modifyTable(htd);
admin.flush(TABLENAME);
@ -213,9 +213,9 @@ public class TestSimpleRegionNormalizerOnCluster {
region.flush(true);
}
HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(tableName));
HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(tableName));
htd.setNormalizationEnabled(true);
admin.modifyTable(tableName, htd);
admin.modifyTable(htd);
admin.flush(tableName);

View File

@ -71,7 +71,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
if (snapshot == null) {
final TableName snapshotTableName = TableName.valueOf("testCloneSnapshot");
long tid = System.currentTimeMillis();
final byte[] snapshotName = Bytes.toBytes("snapshot-" + tid);
final String snapshotName = "snapshot-" + tid;
Admin admin = UTIL.getAdmin();
// create Table

View File

@ -103,7 +103,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
@Test
public void deleteColumnFamilyWithMultipleRegions() throws Exception {
Admin admin = TEST_UTIL.getAdmin();
HTableDescriptor beforehtd = admin.getTableDescriptor(TABLENAME);
HTableDescriptor beforehtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
@ -150,7 +150,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
admin.deleteColumnFamily(TABLENAME, Bytes.toBytes("cf2"));
// 5 - Check if only 2 column families exist in the descriptor
HTableDescriptor afterhtd = admin.getTableDescriptor(TABLENAME);
HTableDescriptor afterhtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
assertEquals(2, afterhtd.getColumnFamilyCount());
HColumnDescriptor[] newFamilies = afterhtd.getColumnFamilies();
assertTrue(newFamilies[0].getNameAsString().equals("cf1"));
@ -181,7 +181,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
@Test
public void deleteColumnFamilyTwice() throws Exception {
Admin admin = TEST_UTIL.getAdmin();
HTableDescriptor beforehtd = admin.getTableDescriptor(TABLENAME);
HTableDescriptor beforehtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
String cfToDelete = "cf1";
FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();

View File

@ -205,7 +205,7 @@ public class TestMasterFailoverWithProcedures {
UTIL.waitUntilAllRegionsAssigned(tableName);
// validate the table regions and layout
regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new RegionInfo[0]);
regions = UTIL.getAdmin().getRegions(tableName).toArray(new RegionInfo[0]);
if (preserveSplits) {
assertEquals(1 + splitKeys.length, regions.length);
} else {

View File

@ -73,7 +73,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
UTIL.getAdmin().disableTable(tableName);
// Modify the table descriptor
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
// Test 1: Modify 1 property
long newMaxFileSize = htd.getMaxFileSize() * 2;
@ -84,7 +84,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
assertEquals(newMaxFileSize, currentHtd.getMaxFileSize());
// Test 2: Modify multiple properties
@ -97,7 +97,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
assertEquals(newReadOnlyOption, currentHtd.isReadOnly());
assertEquals(newMemStoreFlushSize, currentHtd.getMemStoreFlushSize());
}
@ -108,19 +108,19 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "cf1");
HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
assertEquals(1, currentHtd.getFamiliesKeys().size());
// Test 1: Modify the table descriptor online
String cf2 = "cf2";
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
htd.addFamily(new HColumnDescriptor(cf2));
long procId = ProcedureTestingUtility.submitAndWait(
procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
assertEquals(2, currentHtd.getFamiliesKeys().size());
assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
@ -129,7 +129,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
String cf3 = "cf3";
HTableDescriptor htd2 =
new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
htd2.addFamily(new HColumnDescriptor(cf3));
long procId2 =
@ -137,7 +137,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
new ModifyTableProcedure(procExec.getEnvironment(), htd2));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf3)));
assertEquals(3, currentHtd.getFamiliesKeys().size());
}
@ -151,18 +151,18 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, cf2, cf3);
HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
assertEquals(3, currentHtd.getFamiliesKeys().size());
// Test 1: Modify the table descriptor
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
htd.removeFamily(Bytes.toBytes(cf2));
long procId = ProcedureTestingUtility.submitAndWait(
procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
assertEquals(2, currentHtd.getFamiliesKeys().size());
assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf2)));
@ -171,7 +171,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
HTableDescriptor htd2 =
new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
htd2.removeFamily(Bytes.toBytes(cf3));
// Disable Sanity check
htd2.setConfiguration("hbase.table.sanity.checks", Boolean.FALSE.toString());
@ -181,13 +181,13 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
new ModifyTableProcedure(procExec.getEnvironment(), htd2));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
assertEquals(1, currentHtd.getFamiliesKeys().size());
assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf3)));
//Removing the last family will fail
HTableDescriptor htd3 =
new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
htd3.removeFamily(Bytes.toBytes(cf1));
long procId3 =
ProcedureTestingUtility.submitAndWait(procExec,
@ -256,7 +256,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Modify multiple properties of the table.
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
htd.setCompactionEnabled(newCompactionEnableOption);
htd.addFamily(new HColumnDescriptor(cf2));
@ -270,7 +270,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
// Validate descriptor
HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
assertEquals(newCompactionEnableOption, currentHtd.isCompactionEnabled());
assertEquals(2, currentHtd.getFamiliesKeys().size());
assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
@ -554,7 +554,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
public void run() {
try {
UTIL.getAdmin().modifyTable(tableName, htd);
UTIL.getAdmin().modifyTable(htd);
} catch (Exception e) {
if (e.getClass().equals(ConcurrentTableModificationException.class)) {
this.exception = true;

View File

@ -25,9 +25,9 @@ import java.util.Random;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@ -88,7 +88,7 @@ public class TestProcedureAdmin {
public void tearDown() throws Exception {
assertTrue("expected executor to be running", getMasterProcedureExecutor().isRunning());
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
for (HTableDescriptor htd: UTIL.getAdmin().listTables()) {
for (TableDescriptor htd: UTIL.getAdmin().listTableDescriptors()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}

View File

@ -99,7 +99,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
private void setupSnapshotAndUpdateTable() throws Exception {
long tid = System.currentTimeMillis();
final byte[] snapshotName = Bytes.toBytes("snapshot-" + tid);
final String snapshotName = "snapshot-" + tid;
Admin admin = UTIL.getAdmin();
// create Table
SnapshotTestingUtils.createTable(UTIL, snapshotTableName, getNumReplicas(), CF1, CF2);
@ -108,7 +108,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF2, CF2);
SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName, rowCountCF1 + rowCountCF2);
snapshotHTD = admin.getTableDescriptor(snapshotTableName);
snapshotHTD = new HTableDescriptor(admin.getDescriptor(snapshotTableName));
admin.disableTable(snapshotTableName);
// take a snapshot
@ -128,7 +128,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF3, CF3);
SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF4, CF4);
SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF1addition, CF1);
HTableDescriptor currentHTD = admin.getTableDescriptor(snapshotTableName);
HTableDescriptor currentHTD = new HTableDescriptor(admin.getDescriptor(snapshotTableName));
assertTrue(currentHTD.hasFamily(CF1));
assertFalse(currentHTD.hasFamily(CF2));
assertTrue(currentHTD.hasFamily(CF3));
@ -216,7 +216,8 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
try {
UTIL.getAdmin().enableTable(snapshotTableName);
HTableDescriptor currentHTD = UTIL.getAdmin().getTableDescriptor(snapshotTableName);
HTableDescriptor currentHTD =
new HTableDescriptor(UTIL.getAdmin().getDescriptor(snapshotTableName));
assertTrue(currentHTD.hasFamily(CF1));
assertTrue(currentHTD.hasFamily(CF2));
assertFalse(currentHTD.hasFamily(CF3));

View File

@ -18,9 +18,11 @@
package org.apache.hadoop.hbase.master.procedure;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@ -31,8 +33,6 @@ import org.junit.BeforeClass;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.junit.Assert.assertTrue;
public abstract class TestTableDDLProcedureBase {
private static final Logger LOG = LoggerFactory.getLogger(TestTableDDLProcedureBase.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
@ -64,7 +64,7 @@ public abstract class TestTableDDLProcedureBase {
@After
public void tearDown() throws Exception {
resetProcExecutorTestingKillFlag();
for (HTableDescriptor htd: UTIL.getAdmin().listTables()) {
for (TableDescriptor htd : UTIL.getAdmin().listTableDescriptors()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}

View File

@ -101,7 +101,7 @@ public class TestTableDescriptorModificationFromClient {
HTableDescriptor modifiedHtd = new HTableDescriptor(TABLE_NAME);
modifiedHtd.addFamily(new HColumnDescriptor(FAMILY_0));
modifiedHtd.addFamily(new HColumnDescriptor(FAMILY_1));
admin.modifyTable(TABLE_NAME, modifiedHtd);
admin.modifyTable(modifiedHtd);
verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
} finally {
admin.deleteTable(TABLE_NAME);
@ -178,7 +178,7 @@ public class TestTableDescriptorModificationFromClient {
// Modify colymn family
admin.modifyColumnFamily(TABLE_NAME, cfDescriptor);
HTableDescriptor htd = admin.getTableDescriptor(TABLE_NAME);
HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TABLE_NAME));
HColumnDescriptor hcfd = htd.getFamily(FAMILY_0);
assertTrue(hcfd.getBlocksize() == newBlockSize);
} finally {
@ -267,12 +267,12 @@ public class TestTableDescriptorModificationFromClient {
}
}
private void verifyTableDescriptor(final TableName tableName,
final byte[]... families) throws IOException {
private void verifyTableDescriptor(final TableName tableName, final byte[]... families)
throws IOException {
Admin admin = TEST_UTIL.getAdmin();
// Verify descriptor from master
HTableDescriptor htd = admin.getTableDescriptor(tableName);
HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(tableName));
verifyTableDescriptor(htd, tableName, families);
// Verify descriptor from HDFS

View File

@ -92,7 +92,6 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.AfterClass;
import org.junit.Assert;
@ -268,13 +267,7 @@ public class TestMobCompactor {
hcd1.setMobCompactPartitionPolicy(type);
desc.modifyFamily(hcd1);
admin.modifyTable(tableName, desc);
Pair<Integer, Integer> st;
while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
LOG.debug(st.getFirst() + " regions left to update");
Thread.sleep(40);
}
admin.modifyTable(desc);
LOG.info("alter status finished");
}
@ -439,7 +432,7 @@ public class TestMobCompactor {
int rowNumPerRegion = count * rowNumPerFile;
long tid = System.currentTimeMillis();
byte[] snapshotName1 = Bytes.toBytes("snaptb-" + tid);
String snapshotName1 = "snaptb-" + tid;
// take a snapshot
admin.snapshot(snapshotName1, tableName);

View File

@ -42,13 +42,11 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.CompactionState;
import org.apache.hadoop.hbase.client.Connection;
@ -83,8 +81,6 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
@ -130,7 +126,7 @@ public class TestNamespaceAuditor {
@After
public void cleanup() throws Exception, KeeperException {
for (HTableDescriptor table : ADMIN.listTables()) {
for (TableDescriptor table : ADMIN.listTableDescriptors()) {
ADMIN.disableTable(table.getTableName());
deleteTable(table.getTableName());
}
@ -714,8 +710,8 @@ public class TestNamespaceAuditor {
String snapshot = "snapshot_testRestoreSnapshot";
ADMIN.snapshot(snapshot, tableName1);
List<HRegionInfo> regions = ADMIN.getTableRegions(tableName1);
Collections.sort(regions);
List<RegionInfo> regions = ADMIN.getRegions(tableName1);
Collections.sort(regions, RegionInfo.COMPARATOR);
ADMIN.split(tableName1, Bytes.toBytes("JJJ"));
Thread.sleep(2000);

View File

@ -66,7 +66,7 @@ public class TestProcedureManager {
public void testSimpleProcedureManager() throws IOException {
Admin admin = util.getAdmin();
byte[] result = admin.execProcedureWithRet(SimpleMasterProcedureManager.SIMPLE_SIGNATURE,
byte[] result = admin.execProcedureWithReturn(SimpleMasterProcedureManager.SIMPLE_SIGNATURE,
"mytest", new HashMap<>());
assertArrayEquals("Incorrect return data from execProcedure",
Bytes.toBytes(SimpleMasterProcedureManager.SIMPLE_DATA), result);

View File

@ -180,7 +180,7 @@ public class TestTablesWithQuotas {
// Set up Admin to return null (match the implementation)
Admin admin = mock(Admin.class);
when(conn.getAdmin()).thenReturn(admin);
when(admin.getTableRegions(missingTable)).thenReturn(null);
when(admin.getRegions(missingTable)).thenReturn(null);
QuotaObserverChore chore = mock(QuotaObserverChore.class);
Map<RegionInfo,Long> regionUsage = new HashMap<>();

Some files were not shown because too many files have changed in this diff Show More