HBASE-18997 Remove the redundant methods in RegionInfo
This commit is contained in:
parent
5d9b7a978c
commit
240b4b16ff
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -158,7 +158,7 @@ public class TestHRegionInfo {
|
|||
|
||||
@Test
|
||||
public void testMetaTables() {
|
||||
assertTrue(HRegionInfo.FIRST_META_REGIONINFO.isMetaTable());
|
||||
assertTrue(HRegionInfo.FIRST_META_REGIONINFO.isMetaRegion());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -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"})));
|
||||
}
|
||||
|
|
|
@ -189,7 +189,7 @@ public class TestRegionInfoBuilder {
|
|||
|
||||
@Test
|
||||
public void testMetaTables() {
|
||||
assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isMetaTable());
|
||||
assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isMetaRegion());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -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) {
|
||||
|
|
Loading…
Reference in New Issue