HBASE-18997 Remove the redundant methods in RegionInfo

This commit is contained in:
Chia-Ping Tsai 2017-10-14 23:26:11 +08:00
parent 5d9b7a978c
commit 240b4b16ff
27 changed files with 53 additions and 76 deletions

View File

@ -142,7 +142,7 @@ class AsyncRegionLocator {
}
void updateCachedLocation(HRegionLocation loc, Throwable exception) {
if (loc.getRegionInfo().isMetaTable()) {
if (loc.getRegion().isMetaRegion()) {
metaRegionLocator.updateCachedLocation(loc, exception);
} else {
nonMetaRegionLocator.updateCachedLocation(loc, exception);

View File

@ -214,16 +214,6 @@ public interface RegionInfo {
*/
boolean isSplitParent();
/**
* @return true if this region is from hbase:meta.
*/
boolean isMetaTable();
/**
* @return true if this region is from a system table.
*/
boolean isSystemTable();
/**
* @return true if this region is a meta region.
*/

View File

@ -362,28 +362,12 @@ public class RegionInfoBuilder {
Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
}
/**
* @return true if this region is from hbase:meta
*/
@Override
public boolean isMetaTable() {
return isMetaRegion();
}
/** @return true if this region is a meta region */
@Override
public boolean isMetaRegion() {
return tableName.equals(FIRST_META_REGIONINFO.getTable());
}
/**
* @return true if this region is from a system table
*/
@Override
public boolean isSystemTable() {
return tableName.isSystemTable();
}
/**
* @return True if has been split and has daughters.
*/

View File

@ -261,7 +261,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
public void prePut(ObserverContext<RegionCoprocessorEnvironment> e, Put put, WALEdit edit,
Durability durability) throws IOException {
Region region = e.getEnvironment().getRegion();
if (!region.getRegionInfo().isMetaTable()
if (!region.getRegionInfo().isMetaRegion()
&& !region.getRegionInfo().getTable().isSystemTable()) {
if (put.getAttribute(TEST_ATR_KEY) != null) {
LOG.debug("allow any put to happen " + region.getRegionInfo().getRegionNameAsString());

View File

@ -169,7 +169,7 @@ public class TestImportTSVWithTTLs implements Configurable {
public void prePut(ObserverContext<RegionCoprocessorEnvironment> e, Put put, WALEdit edit,
Durability durability) throws IOException {
Region region = e.getEnvironment().getRegion();
if (!region.getRegionInfo().isMetaTable()
if (!region.getRegionInfo().isMetaRegion()
&& !region.getRegionInfo().getTable().isSystemTable()) {
// The put carries the TTL attribute
if (put.getTTL() != Long.MAX_VALUE) {

View File

@ -117,7 +117,7 @@ public class FavoredNodesManager {
* we apply any favored nodes logic on a region.
*/
public static boolean isFavoredNodeApplicable(RegionInfo regionInfo) {
return !regionInfo.isSystemTable();
return !regionInfo.getTable().isSystemTable();
}
/**

View File

@ -1645,7 +1645,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
ServerName dest;
List<ServerName> exclude = hri.isSystemTable() ? assignmentManager.getExcludedServersForSystemTable()
List<ServerName> exclude = hri.getTable().isSystemTable() ? assignmentManager.getExcludedServersForSystemTable()
: new ArrayList<>(1);
if (destServerName != null && exclude.contains(ServerName.valueOf(Bytes.toString(destServerName)))) {
LOG.info(

View File

@ -522,7 +522,7 @@ public class AssignmentManager implements ServerListener {
}
return regions.stream()
.map(RegionStateNode::getRegionInfo)
.filter(RegionInfo::isSystemTable)
.filter(r -> r.getTable().isSystemTable())
.collect(Collectors.toList());
}
@ -691,7 +691,7 @@ public class AssignmentManager implements ServerListener {
}
public MoveRegionProcedure createMoveRegionProcedure(final RegionPlan plan) {
if (plan.getRegionInfo().isSystemTable()) {
if (plan.getRegionInfo().getTable().isSystemTable()) {
List<ServerName> exclude = getExcludedServersForSystemTable();
if (plan.getDestination() != null && exclude.contains(plan.getDestination())) {
try {
@ -1365,7 +1365,7 @@ public class AssignmentManager implements ServerListener {
List<RegionInfo> systemList = new ArrayList<>();
List<RegionInfo> userList = new ArrayList<>();
for (RegionInfo hri : regions) {
if (hri.isSystemTable()) systemList.add(hri);
if (hri.getTable().isSystemTable()) systemList.add(hri);
else userList.add(hri);
}
// Append userList to systemList

View File

@ -1048,7 +1048,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* If so, the primary replica may be expected to be put on the master regionserver.
*/
public boolean shouldBeOnMaster(RegionInfo region) {
return this.onlySystemTablesOnMaster && region.isSystemTable();
return this.onlySystemTablesOnMaster && region.getTable().isSystemTable();
}
/**

View File

@ -157,7 +157,7 @@ class RegionLocationFinder {
boolean includesUserTables = false;
for (final RegionInfo hri : am.getAssignedRegions()) {
cache.refresh(hri);
includesUserTables = includesUserTables || !hri.isSystemTable();
includesUserTables = includesUserTables || !hri.getTable().isSystemTable();
}
return includesUserTables;
}

View File

@ -185,7 +185,7 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
List<Cell> cells = new ArrayList<>();
boolean hasMore;
String flushName = ThroughputControlUtil.getNameForThrottling(store, "flush");
boolean control = throughputController != null && !store.getRegionInfo().isSystemTable();
boolean control = throughputController != null && !store.getRegionInfo().getTable().isSystemTable();
if (control) {
throughputController.start(flushName);
}

View File

@ -218,7 +218,7 @@ public class AnnotationReadingPriorityFunction implements PriorityFunction {
Method getRegion = methodMap.get("getRegion").get(rpcArgClass);
regionSpecifier = (RegionSpecifier)getRegion.invoke(param, (Object[])null);
Region region = rpcServices.getRegion(regionSpecifier);
if (region.getRegionInfo().isSystemTable()) {
if (region.getRegionInfo().getTable().isSystemTable()) {
if (LOG.isTraceEnabled()) {
LOG.trace("High priority because region=" +
region.getRegionInfo().getRegionNameAsString());
@ -239,7 +239,7 @@ public class AnnotationReadingPriorityFunction implements PriorityFunction {
return HConstants.NORMAL_QOS;
}
RegionScanner scanner = rpcServices.getScanner(request.getScannerId());
if (scanner != null && scanner.getRegionInfo().isSystemTable()) {
if (scanner != null && scanner.getRegionInfo().getTable().isSystemTable()) {
if (LOG.isTraceEnabled()) {
// Scanner requests are small in size so TextFormat version should not overwhelm log.
LOG.trace("High priority scanner request " + TextFormat.shortDebugString(request));

View File

@ -2379,7 +2379,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return true;
}
long modifiedFlushCheckInterval = flushCheckInterval;
if (getRegionInfo().isSystemTable() &&
if (getRegionInfo().getTable().isSystemTable() &&
getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
modifiedFlushCheckInterval = SYSTEM_CACHE_FLUSH_INTERVAL;
}
@ -7861,7 +7861,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
*/
public byte[] checkSplit() {
// Can't split META
if (this.getRegionInfo().isMetaTable() ||
if (this.getRegionInfo().isMetaRegion() ||
TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTable())) {
if (shouldForceSplit()) {
LOG.warn("Cannot split meta region in HBase 0.20 and above");
@ -8244,7 +8244,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
buf.append(title + ", ");
buf.append(getRegionInfo().toString());
buf.append(getRegionInfo().isMetaRegion() ? " meta region " : " ");
buf.append(getRegionInfo().isMetaTable() ? " meta table " : " ");
buf.append("stores: ");
for (HStore s : stores.values()) {
buf.append(s.getColumnFamilyDescriptor().getNameAsString());

View File

@ -1236,7 +1236,7 @@ public class HRegionServer extends HasThread implements
if (getNumberOfOnlineRegions() > 2) return false;
boolean allUserRegionsOffline = true;
for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
if (!e.getValue().getRegionInfo().isMetaTable()) {
if (!e.getValue().getRegionInfo().isMetaRegion()) {
allUserRegionsOffline = false;
break;
}
@ -2078,7 +2078,7 @@ public class HRegionServer extends HasThread implements
public WAL getWAL(RegionInfo regionInfo) throws IOException {
WAL wal;
// _ROOT_ and hbase:meta regions have separate WAL.
if (regionInfo != null && regionInfo.isMetaTable()
if (regionInfo != null && regionInfo.isMetaRegion()
&& regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
wal = walFactory.getMetaWAL(regionInfo.getEncodedNameAsBytes());
} else if (regionInfo == null) {
@ -2674,7 +2674,7 @@ public class HRegionServer extends HasThread implements
try {
for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
HRegion r = e.getValue();
if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
if (!r.getRegionInfo().isMetaRegion() && r.isAvailable()) {
// Don't update zk with this close transition; pass false.
closeRegionIgnoreErrors(r.getRegionInfo(), abort);
}

View File

@ -550,7 +550,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
private void mutateRows(final HRegion region,
final List<ClientProtos.Action> actions,
final CellScanner cellScanner, RegionActionResult.Builder builder) throws IOException {
if (!region.getRegionInfo().isMetaTable()) {
if (!region.getRegionInfo().isMetaRegion()) {
regionServer.cacheFlusher.reclaimMemStoreMemory();
}
RowMutations rm = null;
@ -604,7 +604,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
CompareOperator op, ByteArrayComparable comparator, RegionActionResult.Builder builder,
ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
if (!region.getRegionInfo().isMetaTable()) {
if (!region.getRegionInfo().isMetaRegion()) {
regionServer.cacheFlusher.reclaimMemStoreMemory();
}
RowMutations rm = null;
@ -983,7 +983,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
quota.addMutation(mutation);
}
if (!region.getRegionInfo().isMetaTable()) {
if (!region.getRegionInfo().isMetaRegion()) {
regionServer.cacheFlusher.reclaimMemStoreMemory();
}
@ -1095,7 +1095,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
requestCount.increment();
requestRowActionCount.add(mutations.size());
if (!region.getRegionInfo().isMetaTable()) {
if (!region.getRegionInfo().isMetaRegion()) {
regionServer.cacheFlusher.reclaimMemStoreMemory();
}
return region.batchReplay(mutations.toArray(
@ -2697,7 +2697,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
HRegion region = getRegion(request.getRegion());
MutateResponse.Builder builder = MutateResponse.newBuilder();
MutationProto mutation = request.getMutation();
if (!region.getRegionInfo().isMetaTable()) {
if (!region.getRegionInfo().isMetaRegion()) {
regionServer.cacheFlusher.reclaimMemStoreMemory();
}
long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;

View File

@ -119,7 +119,7 @@ abstract class StoreFlusher {
boolean hasMore;
String flushName = ThroughputControlUtil.getNameForThrottling(store, "flush");
// no control on system table (such as meta, namespace, etc) flush
boolean control = throughputController != null && !store.getRegionInfo().isSystemTable();
boolean control = throughputController != null && !store.getRegionInfo().getTable().isSystemTable();
if (control) {
throughputController.start(flushName);
}

View File

@ -91,7 +91,7 @@ public class StorefileRefresherChore extends ScheduledChore {
}
// don't refresh unless enabled for all files, or it the meta region
// meta region don't have WAL replication for replicas enabled yet
if (onlyMetaRefresh && !r.getRegionInfo().isMetaTable()) continue;
if (onlyMetaRefresh && !r.getRegionInfo().isMetaRegion()) continue;
String encodedName = r.getRegionInfo().getEncodedName();
long time = EnvironmentEdgeManager.currentTime();
if (!lastRefreshTimes.containsKey(encodedName)) {

View File

@ -4316,7 +4316,7 @@ public class HBaseFsck extends Configured implements Closeable {
private List<RegionInfo> filterRegions(List<RegionInfo> regions) {
List<RegionInfo> ret = Lists.newArrayList();
for (RegionInfo hri : regions) {
if (hri.isMetaTable() || (!hbck.checkMetaOnly
if (hri.isMetaRegion() || (!hbck.checkMetaOnly
&& hbck.isTableIncluded(hri.getTable()))) {
ret.add(hri);
}

View File

@ -349,7 +349,7 @@ public class TestAdmin2 {
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(tableName));
List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (!regionInfo.isMetaRegion()) {
if (regionInfo.getRegionNameAsString().contains(name)) {
info = regionInfo;
try {
@ -374,7 +374,7 @@ public class TestAdmin2 {
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (!regionInfo.isMetaRegion()) {
if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion2")) {
info = regionInfo;
admin.unassign(regionInfo.getRegionName(), true);

View File

@ -85,7 +85,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
@Override
public RegionScanner preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e, Scan scan,
RegionScanner s) throws IOException {
if (e.getEnvironment().getRegionInfo().isMetaTable()) {
if (e.getEnvironment().getRegionInfo().isMetaRegion()) {
int concurrency = CONCURRENCY.incrementAndGet();
for (;;) {
int max = MAX_CONCURRENCY.get();
@ -104,7 +104,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
@Override
public void postScannerClose(ObserverContext<RegionCoprocessorEnvironment> e, InternalScanner s)
throws IOException {
if (e.getEnvironment().getRegionInfo().isMetaTable()) {
if (e.getEnvironment().getRegionInfo().isMetaRegion()) {
CONCURRENCY.decrementAndGet();
}
}

View File

@ -98,7 +98,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (!regionInfo.isMetaRegion()) {
if (regionInfo.getRegionNameAsString().contains(tableName.getNameAsString())) {
info = regionInfo;
boolean catchNotServingException = false;
@ -125,7 +125,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (RegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (!regionInfo.isMetaRegion()) {
if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegionWhenServerNameIsEmpty")) {
admin.closeRegion(regionInfo.getRegionName(), Optional.empty()).get();
}

View File

@ -1926,7 +1926,7 @@ public class TestMasterObserver {
int moveCnt = openRegions.size()/2;
for (int i=0; i<moveCnt; i++) {
RegionInfo info = openRegions.get(i);
if (!info.isMetaTable()) {
if (!info.isMetaRegion()) {
master.getMasterRpcServices().moveRegion(null, RequestConverter.buildMoveRegionRequest(
openRegions.get(i).getEncodedNameAsBytes(), destRS));
}

View File

@ -836,7 +836,7 @@ public class TestDistributedLogSplitting {
Iterator<RegionInfo> it = regions.iterator();
while (it.hasNext()) {
RegionInfo region = it.next();
if (region.isMetaTable()) {
if (region.isMetaRegion()) {
it.remove();
}
}
@ -1289,7 +1289,7 @@ public class TestDistributedLogSplitting {
Iterator<RegionInfo> it = regions.iterator();
while (it.hasNext()) {
RegionInfo region = it.next();
if (region.isMetaTable()
if (region.isMetaRegion()
|| region.getEncodedName().equals(
RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName())) {
it.remove();
@ -1385,7 +1385,7 @@ public class TestDistributedLogSplitting {
Iterator<RegionInfo> it = regions.iterator();
while (it.hasNext()) {
RegionInfo region = it.next();
if (region.isMetaTable()
if (region.isMetaRegion()
|| region.getEncodedName().equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName())) {
it.remove();
}

View File

@ -158,7 +158,7 @@ public class TestHRegionInfo {
@Test
public void testMetaTables() {
assertTrue(HRegionInfo.FIRST_META_REGIONINFO.isMetaTable());
assertTrue(HRegionInfo.FIRST_META_REGIONINFO.isMetaRegion());
}
@Test

View File

@ -24,6 +24,9 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@ -32,7 +35,6 @@ import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Get;
@ -81,7 +83,7 @@ public class TestPriorityRpc {
RegionSpecifier.Builder regionSpecifierBuilder = RegionSpecifier.newBuilder();
regionSpecifierBuilder.setType(RegionSpecifierType.REGION_NAME);
ByteString name = UnsafeByteOperations.unsafeWrap(
HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName());
regionSpecifierBuilder.setValue(name);
RegionSpecifier regionSpecifier = regionSpecifierBuilder.build();
getRequestBuilder.setRegion(regionSpecifier);
@ -94,10 +96,10 @@ public class TestPriorityRpc {
HRegionServer mockRS = Mockito.mock(HRegionServer.class);
RSRpcServices mockRpc = Mockito.mock(RSRpcServices.class);
Mockito.when(mockRS.getRSRpcServices()).thenReturn(mockRpc);
HRegionInfo mockRegionInfo = Mockito.mock(HRegionInfo.class);
Mockito.when(mockRpc.getRegion((RegionSpecifier) Mockito.any())).thenReturn(mockRegion);
RegionInfo mockRegionInfo = Mockito.mock(RegionInfo.class);
Mockito.when(mockRpc.getRegion(Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRegionInfo.isSystemTable()).thenReturn(true);
Mockito.when(mockRegionInfo.getTable()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable());
// Presume type.
((AnnotationReadingPriorityFunction)priority).setRegionServer(mockRS);
assertEquals(HConstants.SYSTEMTABLE_QOS, priority.getPriority(header, getRequest,
@ -131,10 +133,11 @@ public class TestPriorityRpc {
HRegionServer mockRS = Mockito.mock(HRegionServer.class);
RSRpcServices mockRpc = Mockito.mock(RSRpcServices.class);
Mockito.when(mockRS.getRSRpcServices()).thenReturn(mockRpc);
HRegionInfo mockRegionInfo = Mockito.mock(HRegionInfo.class);
Mockito.when(mockRpc.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
RegionInfo mockRegionInfo = Mockito.mock(RegionInfo.class);
Mockito.when(mockRpc.getRegion(Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRegionInfo.isSystemTable()).thenReturn(false);
// make isSystemTable return false
Mockito.when(mockRegionInfo.getTable()).thenReturn(TableName.valueOf("testQosFunctionForScanMethod"));
// Presume type.
((AnnotationReadingPriorityFunction)priority).setRegionServer(mockRS);
int qos = priority.getPriority(header, scanRequest,
@ -151,7 +154,7 @@ public class TestPriorityRpc {
Mockito.when(mockRegionScanner.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRpc.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRegionInfo.isSystemTable()).thenReturn(true);
Mockito.when(mockRegionInfo.getTable()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable());
// Presume type.
((AnnotationReadingPriorityFunction)priority).setRegionServer(mockRS);
@ -160,7 +163,8 @@ public class TestPriorityRpc {
User.createUserForTesting(regionServer.conf, "someuser", new String[]{"somegroup"})));
//the same as above but with non-meta region
Mockito.when(mockRegionInfo.isSystemTable()).thenReturn(false);
// make isSystemTable return false
Mockito.when(mockRegionInfo.getTable()).thenReturn(TableName.valueOf("testQosFunctionForScanMethod"));
assertEquals(HConstants.NORMAL_QOS, priority.getPriority(header, scanRequest,
User.createUserForTesting(regionServer.conf, "someuser", new String[]{"somegroup"})));
}

View File

@ -189,7 +189,7 @@ public class TestRegionInfoBuilder {
@Test
public void testMetaTables() {
assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isMetaTable());
assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isMetaRegion());
}
@Test

View File

@ -149,7 +149,7 @@ public class BaseTestHBaseFsck {
RegionInfo hri) throws IOException, InterruptedException {
try {
HBaseFsckRepair.closeRegionSilentlyAndWait(conn, sn, hri);
if (!hri.isMetaTable()) {
if (!hri.isMetaRegion()) {
admin.offline(hri.getRegionName());
}
} catch (IOException ioe) {