HBASE-24964 Remove MetaTableAccessor.tableExists (#2330)

Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
Duo Zhang 2020-08-30 17:31:26 +08:00 committed by GitHub
parent ef5da4a61d
commit 2c4204ad75
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 101 additions and 144 deletions

View File

@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.backup.HBackupFileSystem;
import org.apache.hadoop.hbase.backup.RestoreRequest;
import org.apache.hadoop.hbase.backup.impl.BackupManifest;
import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
@ -128,32 +129,34 @@ public final class BackupUtils {
// for each table in the table set, copy out the table info and region
// info files in the correct directory structure
for (TableName table : backupInfo.getTables()) {
if (!MetaTableAccessor.tableExists(conn, table)) {
LOG.warn("Table " + table + " does not exists, skipping it.");
continue;
}
TableDescriptor orig = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, table);
try (Admin admin = conn.getAdmin()) {
for (TableName table : backupInfo.getTables()) {
if (!admin.tableExists(table)) {
LOG.warn("Table " + table + " does not exists, skipping it.");
continue;
}
TableDescriptor orig = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, table);
// write a copy of descriptor to the target directory
Path target = new Path(backupInfo.getTableBackupDir(table));
FileSystem targetFs = target.getFileSystem(conf);
FSTableDescriptors descriptors =
new FSTableDescriptors(targetFs, CommonFSUtils.getRootDir(conf));
descriptors.createTableDescriptorForTableDirectory(target, orig, false);
LOG.debug("Attempting to copy table info for:" + table + " target: " + target
+ " descriptor: " + orig);
LOG.debug("Finished copying tableinfo.");
List<RegionInfo> regions = MetaTableAccessor.getTableRegions(conn, table);
// For each region, write the region info to disk
LOG.debug("Starting to write region info for table " + table);
for (RegionInfo regionInfo : regions) {
Path regionDir = FSUtils
.getRegionDirFromTableDir(new Path(backupInfo.getTableBackupDir(table)), regionInfo);
regionDir = new Path(backupInfo.getTableBackupDir(table), regionDir.getName());
writeRegioninfoOnFilesystem(conf, targetFs, regionDir, regionInfo);
// write a copy of descriptor to the target directory
Path target = new Path(backupInfo.getTableBackupDir(table));
FileSystem targetFs = target.getFileSystem(conf);
FSTableDescriptors descriptors =
new FSTableDescriptors(targetFs, CommonFSUtils.getRootDir(conf));
descriptors.createTableDescriptorForTableDirectory(target, orig, false);
LOG.debug("Attempting to copy table info for:" + table + " target: " + target +
" descriptor: " + orig);
LOG.debug("Finished copying tableinfo.");
List<RegionInfo> regions = MetaTableAccessor.getTableRegions(conn, table);
// For each region, write the region info to disk
LOG.debug("Starting to write region info for table " + table);
for (RegionInfo regionInfo : regions) {
Path regionDir = FSUtils
.getRegionDirFromTableDir(new Path(backupInfo.getTableBackupDir(table)), regionInfo);
regionDir = new Path(backupInfo.getTableBackupDir(table), regionDir.getName());
writeRegioninfoOnFilesystem(conf, targetFs, regionDir, regionInfo);
}
LOG.debug("Finished writing region info for table " + table);
}
LOG.debug("Finished writing region info for table " + table);
}
}

View File

@ -354,20 +354,6 @@ public final class MetaTableAccessor {
PrivateCellUtil.qualifierStartsWith(cell, HConstants.MERGE_QUALIFIER_PREFIX);
}
/**
* Checks if the specified table exists. Looks at the hbase:meta table hosted on the specified
* server.
* @param connection connection we're using
* @param tableName table to check
* @return true if the table exists in meta, false if not
*/
public static boolean tableExists(Connection connection, final TableName tableName)
throws IOException {
// Catalog tables always exist.
return tableName.equals(TableName.META_TABLE_NAME) ||
getTableState(connection, tableName) != null;
}
/**
* Lists all of the regions currently in META.
* @param connection to connect with

View File

@ -28,6 +28,14 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
*/
@InterfaceAudience.Private
public interface TableDescriptors {
/**
* Test whether a given table exists, i.e, has a table descriptor.
*/
default boolean exists(TableName tableName) throws IOException {
return get(tableName) != null;
}
/**
* @return TableDescriptor for tablename
*/

View File

@ -2607,8 +2607,8 @@ public class HMaster extends HRegionServer implements MasterServices {
}
private void checkTableExists(final TableName tableName)
throws IOException, TableNotFoundException {
if (!MetaTableAccessor.tableExists(getConnection(), tableName)) {
throws IOException, TableNotFoundException {
if (!tableDescriptors.exists(tableName)) {
throw new TableNotFoundException(tableName);
}
}

View File

@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.RegionInfo;
@ -78,19 +76,6 @@ public abstract class AbstractStateMachineRegionProcedure<TState>
sb.append(", region=").append(getRegion().getShortNameToLog());
}
/**
* Check whether a table is modifiable - exists and either offline or online with config set
* @param env MasterProcedureEnv
* @throws IOException
*/
@Override
protected void checkTableModifiable(final MasterProcedureEnv env) throws IOException {
// Checks whether the table exists
if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), getTableName())) {
throw new TableNotFoundException(getTableName());
}
}
@Override
protected boolean holdLock(MasterProcedureEnv env) {
return true;

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotEnabledException;
@ -120,11 +119,10 @@ public abstract class AbstractStateMachineTableProcedure<TState>
/**
* Check whether a table is modifiable - exists and either offline or online with config set
* @param env MasterProcedureEnv
* @throws IOException
*/
protected void checkTableModifiable(final MasterProcedureEnv env) throws IOException {
// Checks whether the table exists
if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), getTableName())) {
if (!env.getMasterServices().getTableDescriptors().exists(getTableName())) {
throw new TableNotFoundException(getTableName());
}
}

View File

@ -27,7 +27,6 @@ import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
@ -330,12 +329,11 @@ public class CloneSnapshotProcedure
/**
* Action before any real action of cloning from snapshot.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void prepareClone(final MasterProcedureEnv env) throws IOException {
final TableName tableName = getTableName();
if (MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
throw new TableExistsException(getTableName());
if (env.getMasterServices().getTableDescriptors().exists(tableName)) {
throw new TableExistsException(tableName);
}
}

View File

@ -243,7 +243,7 @@ public class CreateTableProcedure
private boolean prepareCreate(final MasterProcedureEnv env) throws IOException {
final TableName tableName = getTableName();
if (MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
if (env.getMasterServices().getTableDescriptors().exists(tableName)) {
setFailure("master-create-table", new TableExistsException(getTableName()));
return false;
}

View File

@ -246,7 +246,7 @@ public class DisableTableProcedure
setFailure("master-disable-table",
new ConstraintException("Cannot disable " + this.tableName));
canTableBeDisabled = false;
} else if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
} else if (!env.getMasterServices().getTableDescriptors().exists(tableName)) {
setFailure("master-disable-table", new TableNotFoundException(tableName));
canTableBeDisabled = false;
} else if (!skipTableStateCheck) {

View File

@ -282,9 +282,8 @@ public class EnableTableProcedure
*/
private boolean prepareEnable(final MasterProcedureEnv env) throws IOException {
boolean canTableBeEnabled = true;
// Check whether table exists
if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
if (!env.getMasterServices().getTableDescriptors().exists(tableName)) {
setFailure("master-enable-table", new TableNotFoundException(tableName));
canTableBeEnabled = false;
} else {

View File

@ -274,7 +274,7 @@ public class ModifyTableProcedure
*/
private void prepareModify(final MasterProcedureEnv env) throws IOException {
// Checks whether the table exists
if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), getTableName())) {
if (!env.getMasterServices().getTableDescriptors().exists(getTableName())) {
throw new TableNotFoundException(getTableName());
}

View File

@ -24,7 +24,6 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException;
@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@ -330,7 +330,7 @@ public class RestoreSnapshotProcedure
private void prepareRestore(final MasterProcedureEnv env) throws IOException {
final TableName tableName = getTableName();
// Checks whether the table exists
if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
if (!env.getMasterServices().getTableDescriptors().exists(tableName)) {
throw new TableNotFoundException(tableName);
}

View File

@ -63,7 +63,7 @@ public class SlowLogMasterService {
LOG.info("Slow/Large requests logging to system table hbase:slowlog is disabled. Quitting.");
return;
}
if (!MetaTableAccessor.tableExists(masterServices.getConnection(),
if (!masterServices.getTableDescriptors().exists(
SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)) {
LOG.info("slowlog table not found. Creating.");
this.masterServices.createSystemTable(TABLE_DESCRIPTOR_BUILDER.build());

View File

@ -42,7 +42,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.TableDescriptor;
@ -851,7 +850,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
// Execute the restore/clone operation
long procId;
if (MetaTableAccessor.tableExists(master.getConnection(), tableName)) {
if (master.getTableDescriptors().exists(tableName)) {
procId = restoreSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey,
restoreAcl);
} else {

View File

@ -66,7 +66,7 @@ public class NamespaceAuditor {
public void checkQuotaToCreateTable(TableName tName, int regions) throws IOException {
if (stateManager.isInitialized()) {
// We do this check to fail fast.
if (MetaTableAccessor.tableExists(this.masterServices.getConnection(), tName)) {
if (masterServices.getTableDescriptors().exists(tName)) {
throw new TableExistsException(tName);
}
stateManager.checkAndUpdateNamespaceTableCount(tName, regions);

View File

@ -28,12 +28,10 @@ import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionStateListener;
import org.apache.hadoop.hbase.TableName;
@ -105,8 +103,7 @@ public class MasterQuotaManager implements RegionStateListener {
}
// Create the quota table if missing
if (!MetaTableAccessor.tableExists(masterServices.getConnection(),
QuotaUtil.QUOTA_TABLE_NAME)) {
if (!masterServices.getTableDescriptors().exists(QuotaUtil.QUOTA_TABLE_NAME)) {
LOG.info("Quota table not found. Creating...");
createQuotaTable();
}

View File

@ -21,19 +21,18 @@ import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.TableName;
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.Connection;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A {@link ScheduledChore} which periodically updates the {@link RegionServerSpaceQuotaManager}
@ -160,7 +159,9 @@ public class SpaceQuotaRefresherChore extends ScheduledChore {
* @throws IOException throws IOException
*/
boolean checkQuotaTableExists() throws IOException {
return MetaTableAccessor.tableExists(getConnection(), QuotaUtil.QUOTA_TABLE_NAME);
try (Admin admin = getConnection().getAdmin()) {
return admin.tableExists(QuotaUtil.QUOTA_TABLE_NAME);
}
}
/**

View File

@ -122,6 +122,6 @@ public class SnapshotScannerHDFSAclCleaner extends BaseHFileCleanerDelegate {
}
private boolean tableExists(TableName tableName) throws IOException {
return master != null && MetaTableAccessor.tableExists(master.getConnection(), tableName);
return master != null && master.getTableDescriptors().exists(tableName);
}
}

View File

@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
@ -207,18 +206,20 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso
/********************************* Master related hooks **********************************/
@Override
public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
// Need to create the new system table for labels here
if (!MetaTableAccessor.tableExists(ctx.getEnvironment().getConnection(), LABELS_TABLE_NAME)) {
// We will cache all the labels. No need of normal table block cache.
// Let the "labels" table having only one region always. We are not expecting too many labels
// in the system.
TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(LABELS_TABLE_NAME)
.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(LABELS_TABLE_FAMILY)
.setBloomFilterType(BloomType.NONE).setBlockCacheEnabled(false).build())
.setValue(TableDescriptorBuilder.SPLIT_POLICY, DisabledRegionSplitPolicy.class.getName())
.build();
try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) {
try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) {
if (!admin.tableExists(LABELS_TABLE_NAME)) {
// We will cache all the labels. No need of normal table block cache.
// Let the "labels" table having only one region always. We are not expecting too many
// labels in the system.
TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(LABELS_TABLE_NAME)
.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(LABELS_TABLE_FAMILY)
.setBloomFilterType(BloomType.NONE).setBlockCacheEnabled(false).build())
.setValue(TableDescriptorBuilder.SPLIT_POLICY, DisabledRegionSplitPolicy.class.getName())
.build();
admin.createTable(tableDescriptor);
}
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
@ -323,24 +322,6 @@ public class TestMetaTableAccessor {
assertTrue(MetaTableLocator.getMetaRegionsAndLocations(UTIL.getZooKeeperWatcher()).size() >= 1);
}
@Test
public void testTableExists() throws IOException {
final TableName tableName = TableName.valueOf(name.getMethodName());
assertFalse(MetaTableAccessor.tableExists(connection, tableName));
UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
assertTrue(MetaTableAccessor.tableExists(connection, tableName));
Admin admin = UTIL.getAdmin();
admin.disableTable(tableName);
admin.deleteTable(tableName);
assertFalse(MetaTableAccessor.tableExists(connection, tableName));
assertTrue(MetaTableAccessor.tableExists(connection, TableName.META_TABLE_NAME));
UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
assertTrue(MetaTableAccessor.tableExists(connection, tableName));
admin.disableTable(tableName);
admin.deleteTable(tableName);
assertFalse(MetaTableAccessor.tableExists(connection, tableName));
}
@Test
public void testGetRegion() throws IOException, InterruptedException {
final String name = this.name.getMethodName();

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.RegionLocations;
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.BufferedMutator;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
@ -213,7 +214,7 @@ public class MasterProcedureTestingUtility {
LOG.debug("Table directory layout is as expected.");
// check meta
assertTrue(MetaTableAccessor.tableExists(master.getConnection(), tableName));
assertTrue(tableExists(master.getConnection(), tableName));
assertEquals(regions.length, countMetaRegions(master, tableName));
// check htd
@ -234,7 +235,7 @@ public class MasterProcedureTestingUtility {
assertFalse(fs.exists(tableDir));
// check meta
assertFalse(MetaTableAccessor.tableExists(master.getConnection(), tableName));
assertFalse(tableExists(master.getConnection(), tableName));
assertEquals(0, countMetaRegions(master, tableName));
// check htd
@ -582,6 +583,12 @@ public class MasterProcedureTestingUtility {
}
}
public static boolean tableExists(Connection conn, TableName tableName) throws IOException {
try (Admin admin = conn.getAdmin()) {
return admin.tableExists(tableName);
}
}
public static class InjectAbortOnLoadListener
implements ProcedureExecutor.ProcedureExecutorListener {
private final ProcedureExecutor<MasterProcedureEnv> procExec;

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@ -37,12 +36,9 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog;
import org.apache.hadoop.hbase.slowlog.SlowLogTableAccessor;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
@ -53,6 +49,11 @@ import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog;
/**
* Tests for SlowLog System Table
*/
@ -167,17 +168,16 @@ public class TestSlowLogAccessor {
}
}
private Connection waitForSlowLogTableCreation() {
Connection connection =
HBASE_TESTING_UTILITY.getMiniHBaseCluster().getRegionServer(0).getConnection();
private Connection waitForSlowLogTableCreation() throws IOException {
Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(2000, () -> {
try {
return MetaTableAccessor.tableExists(connection, SlowLogTableAccessor.SLOW_LOG_TABLE_NAME);
return HBASE_TESTING_UTILITY.getAdmin()
.tableExists(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME);
} catch (IOException e) {
return false;
}
}));
return connection;
return HBASE_TESTING_UTILITY.getConnection();
}
@Test

View File

@ -19,17 +19,16 @@ package org.apache.hadoop.hbase.quotas;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Admin;
@ -43,7 +42,6 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
@ -96,9 +94,9 @@ public class TestQuotaObserverChoreRegionReports {
TEST_UTIL.startMiniCluster(1);
// Wait till quota table onlined.
TEST_UTIL.waitFor(10000, new Waiter.Predicate<Exception>() {
@Override public boolean evaluate() throws Exception {
return MetaTableAccessor.tableExists(TEST_UTIL.getConnection(),
QuotaTableUtil.QUOTA_TABLE_NAME);
@Override
public boolean evaluate() throws Exception {
return TEST_UTIL.getAdmin().tableExists(QuotaTableUtil.QUOTA_TABLE_NAME);
}
});
@ -155,9 +153,9 @@ public class TestQuotaObserverChoreRegionReports {
TEST_UTIL.startMiniCluster(1);
// Wait till quota table onlined.
TEST_UTIL.waitFor(10000, new Waiter.Predicate<Exception>() {
@Override public boolean evaluate() throws Exception {
return MetaTableAccessor.tableExists(TEST_UTIL.getConnection(),
QuotaTableUtil.QUOTA_TABLE_NAME);
@Override
public boolean evaluate() throws Exception {
return TEST_UTIL.getAdmin().tableExists(QuotaTableUtil.QUOTA_TABLE_NAME);
}
});

View File

@ -30,7 +30,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.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.Waiter.Predicate;
@ -88,9 +87,9 @@ public class TestSpaceQuotasWithSnapshots {
TEST_UTIL.startMiniCluster(1);
// Wait till quota table onlined.
TEST_UTIL.waitFor(10000, new Waiter.Predicate<Exception>() {
@Override public boolean evaluate() throws Exception {
return MetaTableAccessor.tableExists(TEST_UTIL.getConnection(),
QuotaTableUtil.QUOTA_TABLE_NAME);
@Override
public boolean evaluate() throws Exception {
return TEST_UTIL.getAdmin().tableExists(QuotaTableUtil.QUOTA_TABLE_NAME);
}
});
}

View File

@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.StartMiniClusterOption;
@ -672,8 +671,7 @@ public class TestSplitTransactionOnCluster {
admin.balancerSwitch(false, true);
// Turn off the meta scanner so it don't remove parent on us.
cluster.getMaster().setCatalogJanitorEnabled(false);
boolean tableExists = MetaTableAccessor.tableExists(regionServer.getConnection(),
tableName);
boolean tableExists = TESTING_UTIL.getAdmin().tableExists(tableName);
assertEquals("The specified table should be present.", true, tableExists);
final HRegion region = findSplittableRegion(oldRegions);
regionServerIndex = cluster.getServerWith(region.getRegionInfo().getRegionName());
@ -693,8 +691,7 @@ public class TestSplitTransactionOnCluster {
Thread.sleep(1000);
} while ((newRegions.contains(oldRegions.get(0)) || newRegions.contains(oldRegions.get(1)))
|| newRegions.size() != 4);
tableExists = MetaTableAccessor.tableExists(regionServer.getConnection(),
tableName);
tableExists = TESTING_UTIL.getAdmin().tableExists(tableName);
assertEquals("The specified table should be present.", true, tableExists);
// exists works on stale and we see the put after the flush
byte[] b1 = Bytes.toBytes("row1");