HBASE-18298 RegionServerServices Interface cleanup for CP expose.
This commit is contained in:
parent
417dad646b
commit
bd68551f26
|
@ -19,7 +19,6 @@
|
|||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Interface to support the aborting of a given server or client.
|
||||
|
@ -29,8 +28,7 @@ import org.apache.yetus.audience.InterfaceStability;
|
|||
* <p>
|
||||
* Implemented by the Master, RegionServer, and TableServers (client).
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
@InterfaceAudience.Private
|
||||
public interface Abortable {
|
||||
/**
|
||||
* Abort the server or client.
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Re
|
|||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
|
@ -62,10 +63,13 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
|
|||
private static final Log LOG = LogFactory.getLog(SecureBulkLoadEndpoint.class);
|
||||
|
||||
private RegionCoprocessorEnvironment env;
|
||||
private RegionServerServices rsServices;
|
||||
|
||||
@Override
|
||||
public void start(CoprocessorEnvironment env) {
|
||||
this.env = (RegionCoprocessorEnvironment)env;
|
||||
assert this.env.getCoprocessorRegionServerServices() instanceof RegionServerServices;
|
||||
rsServices = (RegionServerServices) this.env.getCoprocessorRegionServerServices();
|
||||
LOG.warn("SecureBulkLoadEndpoint is deprecated. It will be removed in future releases.");
|
||||
LOG.warn("Secure bulk load has been integrated into HBase core.");
|
||||
}
|
||||
|
@ -78,8 +82,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
|
|||
public void prepareBulkLoad(RpcController controller, PrepareBulkLoadRequest request,
|
||||
RpcCallback<PrepareBulkLoadResponse> done) {
|
||||
try {
|
||||
SecureBulkLoadManager secureBulkLoadManager =
|
||||
this.env.getRegionServerServices().getSecureBulkLoadManager();
|
||||
SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager();
|
||||
|
||||
String bulkToken = secureBulkLoadManager.prepareBulkLoad(this.env.getRegion(),
|
||||
convert(request));
|
||||
|
@ -106,8 +109,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
|
|||
public void cleanupBulkLoad(RpcController controller, CleanupBulkLoadRequest request,
|
||||
RpcCallback<CleanupBulkLoadResponse> done) {
|
||||
try {
|
||||
SecureBulkLoadManager secureBulkLoadManager =
|
||||
this.env.getRegionServerServices().getSecureBulkLoadManager();
|
||||
SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager();
|
||||
secureBulkLoadManager.cleanupBulkLoad(this.env.getRegion(), convert(request));
|
||||
done.run(CleanupBulkLoadResponse.newBuilder().build());
|
||||
} catch (IOException e) {
|
||||
|
@ -138,8 +140,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
|
|||
boolean loaded = false;
|
||||
Map<byte[], List<Path>> map = null;
|
||||
try {
|
||||
SecureBulkLoadManager secureBulkLoadManager =
|
||||
this.env.getRegionServerServices().getSecureBulkLoadManager();
|
||||
SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager();
|
||||
BulkLoadHFileRequest bulkLoadHFileRequest = ConvertSecureBulkLoadHFilesRequest(request);
|
||||
map = secureBulkLoadManager.secureBulkLoadHFiles(this.env.getRegion(),
|
||||
convert(bulkLoadHFileRequest));
|
||||
|
|
|
@ -65,10 +65,17 @@ import org.apache.zookeeper.ZooKeeper;
|
|||
* listeners registered with ZooKeeperWatcher cannot be removed.
|
||||
*/
|
||||
public class ZooKeeperScanPolicyObserver implements RegionObserver {
|
||||
// The zk ensemble info is put in hbase config xml with given custom key.
|
||||
public static final String ZK_ENSEMBLE_KEY = "ZooKeeperScanPolicyObserver.zookeeper.ensemble";
|
||||
public static final String ZK_SESSION_TIMEOUT_KEY =
|
||||
"ZooKeeperScanPolicyObserver.zookeeper.session.timeout";
|
||||
public static final int ZK_SESSION_TIMEOUT_DEFAULT = 30 * 1000; // 30 secs
|
||||
public static final String node = "/backup/example/lastbackup";
|
||||
public static final String zkkey = "ZK";
|
||||
private static final Log LOG = LogFactory.getLog(ZooKeeperScanPolicyObserver.class);
|
||||
|
||||
private ZooKeeper zk = null;
|
||||
|
||||
/**
|
||||
* Internal watcher that keep "data" up to date asynchronously.
|
||||
*/
|
||||
|
@ -165,8 +172,22 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
|
|||
if (!re.getSharedData().containsKey(zkkey)) {
|
||||
// there is a short race here
|
||||
// in the worst case we create a watcher that will be notified once
|
||||
re.getSharedData().putIfAbsent(zkkey, new ZKWatcher(
|
||||
re.getRegionServerServices().getZooKeeper().getRecoverableZooKeeper().getZooKeeper()));
|
||||
String ensemble = re.getConfiguration().get(ZK_ENSEMBLE_KEY);
|
||||
int sessionTimeout = re.getConfiguration().getInt(ZK_SESSION_TIMEOUT_KEY,
|
||||
ZK_SESSION_TIMEOUT_DEFAULT);
|
||||
this.zk = new ZooKeeper(ensemble, sessionTimeout, null);
|
||||
re.getSharedData().putIfAbsent(zkkey, new ZKWatcher(zk));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop(CoprocessorEnvironment env) throws IOException {
|
||||
if (this.zk != null) {
|
||||
try {
|
||||
this.zk.close();
|
||||
} catch (InterruptedException e) {
|
||||
LOG.error("Excepion while closing the ZK connection!", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -740,7 +740,7 @@ public class TestImportExport {
|
|||
|
||||
// Register the wal listener for the import table
|
||||
HRegionInfo region = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer()
|
||||
.getOnlineRegions(importTable.getName()).get(0).getRegionInfo();
|
||||
.getRegions(importTable.getName()).get(0).getRegionInfo();
|
||||
TableWALActionListener walListener = new TableWALActionListener(region);
|
||||
WAL wal = UTIL.getMiniHBaseCluster().getRegionServer(0).getWAL(region);
|
||||
wal.registerWALActionsListener(walListener);
|
||||
|
@ -759,7 +759,7 @@ public class TestImportExport {
|
|||
importTableName = name.getMethodName() + "import2";
|
||||
importTable = UTIL.createTable(TableName.valueOf(importTableName), FAMILYA, 3);
|
||||
region = UTIL.getHBaseCluster().getRegionServerThreads().get(0).getRegionServer()
|
||||
.getOnlineRegions(importTable.getName()).get(0).getRegionInfo();
|
||||
.getRegions(importTable.getName()).get(0).getRegionInfo();
|
||||
wal = UTIL.getMiniHBaseCluster().getRegionServer(0).getWAL(region);
|
||||
walListener = new TableWALActionListener(region);
|
||||
wal.registerWALActionsListener(walListener);
|
||||
|
|
|
@ -176,10 +176,10 @@ public class TestRSGroupsOfflineMode {
|
|||
TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
|
||||
@Override
|
||||
public boolean evaluate() throws Exception {
|
||||
return failoverRS.getOnlineRegions(failoverTable).size() >= 1;
|
||||
return failoverRS.getRegions(failoverTable).size() >= 1;
|
||||
}
|
||||
});
|
||||
Assert.assertEquals(0, failoverRS.getOnlineRegions(RSGroupInfoManager.RSGROUP_TABLE_NAME).size());
|
||||
Assert.assertEquals(0, failoverRS.getRegions(RSGroupInfoManager.RSGROUP_TABLE_NAME).size());
|
||||
|
||||
// Need this for minicluster to shutdown cleanly.
|
||||
master.stopMaster();
|
||||
|
|
|
@ -19,19 +19,17 @@
|
|||
package org.apache.hadoop.hbase;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Defines the set of shared functions implemented by HBase servers (Masters
|
||||
* and RegionServers).
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
@InterfaceAudience.Private
|
||||
public interface Server extends Abortable, Stoppable {
|
||||
/**
|
||||
* Gets the configuration object for this server.
|
||||
|
|
|
@ -25,8 +25,8 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.regionserver.CoprocessorRegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
|
||||
/**
|
||||
|
@ -54,7 +54,7 @@ public class CoprocessorHConnection extends ConnectionImplementation {
|
|||
// this bit is a little hacky - just trying to get it going for the moment
|
||||
if (env instanceof RegionCoprocessorEnvironment) {
|
||||
RegionCoprocessorEnvironment e = (RegionCoprocessorEnvironment) env;
|
||||
RegionServerServices services = e.getRegionServerServices();
|
||||
CoprocessorRegionServerServices services = e.getCoprocessorRegionServerServices();
|
||||
if (services instanceof HRegionServer) {
|
||||
return new CoprocessorHConnection((HRegionServer) services);
|
||||
}
|
||||
|
|
|
@ -25,8 +25,8 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
|||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.metrics.MetricRegistry;
|
||||
import org.apache.hadoop.hbase.regionserver.CoprocessorRegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
|
@ -40,7 +40,7 @@ public interface RegionCoprocessorEnvironment extends CoprocessorEnvironment {
|
|||
RegionInfo getRegionInfo();
|
||||
|
||||
/** @return reference to the region server services */
|
||||
RegionServerServices getRegionServerServices();
|
||||
CoprocessorRegionServerServices getCoprocessorRegionServerServices();
|
||||
|
||||
/** @return shared data between all instances of this coprocessor */
|
||||
ConcurrentMap<String, Object> getSharedData();
|
||||
|
|
|
@ -19,16 +19,21 @@
|
|||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.metrics.MetricRegistry;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.CoprocessorRegionServerServices;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public interface RegionServerCoprocessorEnvironment extends CoprocessorEnvironment {
|
||||
/**
|
||||
* Gets the region server services.
|
||||
*
|
||||
* @return the region server services
|
||||
*/
|
||||
RegionServerServices getRegionServerServices();
|
||||
CoprocessorRegionServerServices getCoprocessorRegionServerServices();
|
||||
|
||||
/**
|
||||
* Returns a MetricRegistry that can be used to track metrics at the region server level.
|
||||
|
|
|
@ -175,7 +175,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur
|
|||
* @throws IOException
|
||||
*/
|
||||
private List<Region> getRegionsToFlush(String table) throws IOException {
|
||||
return rss.getOnlineRegions(TableName.valueOf(table));
|
||||
return rss.getRegions(TableName.valueOf(table));
|
||||
}
|
||||
|
||||
public class FlushTableSubprocedureBuilder implements SubprocedureFactory {
|
||||
|
|
|
@ -77,7 +77,7 @@ public class FileSystemUtilizationChore extends ScheduledChore {
|
|||
return;
|
||||
}
|
||||
final Map<HRegionInfo,Long> onlineRegionSizes = new HashMap<>();
|
||||
final Set<Region> onlineRegions = new HashSet<>(rs.getOnlineRegions());
|
||||
final Set<Region> onlineRegions = new HashSet<>(rs.getRegions());
|
||||
// Process the regions from the last run if we have any. If we are somehow having difficulty
|
||||
// processing the Regions, we want to avoid creating a backlog in memory of Region objs.
|
||||
Iterator<Region> oldRegionsToProcess = getLeftoverRegions();
|
||||
|
|
|
@ -89,7 +89,7 @@ public class CompactedHFilesDischarger extends ScheduledChore {
|
|||
// Noop if rss is null. This will never happen in a normal condition except for cases
|
||||
// when the test case is not spinning up a cluster
|
||||
if (regionServerServices == null) return;
|
||||
List<Region> onlineRegions = regionServerServices.getOnlineRegions();
|
||||
List<Region> onlineRegions = regionServerServices.getRegions();
|
||||
if (onlineRegions == null) return;
|
||||
for (Region region : onlineRegions) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
|
|
|
@ -0,0 +1,64 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Services exposed to CPs by {@link HRegionServer}
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public interface CoprocessorRegionServerServices extends ImmutableOnlineRegions {
|
||||
|
||||
/**
|
||||
* @return True if this regionserver is stopping.
|
||||
*/
|
||||
boolean isStopping();
|
||||
|
||||
/**
|
||||
* @return Return the FileSystem object used by the regionserver
|
||||
*/
|
||||
FileSystem getFileSystem();
|
||||
|
||||
/**
|
||||
* @return all the online tables in this RS
|
||||
*/
|
||||
Set<TableName> getOnlineTables();
|
||||
|
||||
/**
|
||||
* Returns a reference to the servers' connection.
|
||||
*
|
||||
* Important note: this method returns a reference to Connection which is managed
|
||||
* by Server itself, so callers must NOT attempt to close connection obtained.
|
||||
*/
|
||||
Connection getConnection();
|
||||
|
||||
/**
|
||||
* @return The unique server name for this server.
|
||||
*/
|
||||
ServerName getServerName();
|
||||
}
|
|
@ -19,19 +19,16 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.List;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Abstraction that allows different modules in RegionServer to update/get
|
||||
* the favored nodes information for regions.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
@InterfaceAudience.Private
|
||||
public interface FavoredNodesForRegion {
|
||||
/**
|
||||
* Used to update the favored nodes mapping when required.
|
||||
|
|
|
@ -2757,7 +2757,7 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public void addToOnlineRegions(Region region) {
|
||||
public void addRegion(Region region) {
|
||||
this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
|
||||
configurationManager.registerObserver(region);
|
||||
}
|
||||
|
@ -3003,7 +3003,7 @@ public class HRegionServer extends HasThread implements
|
|||
* @return Online regions from <code>tableName</code>
|
||||
*/
|
||||
@Override
|
||||
public List<Region> getOnlineRegions(TableName tableName) {
|
||||
public List<Region> getRegions(TableName tableName) {
|
||||
List<Region> tableRegions = new ArrayList<>();
|
||||
synchronized (this.onlineRegions) {
|
||||
for (Region region: this.onlineRegions.values()) {
|
||||
|
@ -3017,7 +3017,7 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<Region> getOnlineRegions() {
|
||||
public List<Region> getRegions() {
|
||||
List<Region> allRegions = new ArrayList<>();
|
||||
synchronized (this.onlineRegions) {
|
||||
// Return a clone copy of the onlineRegions
|
||||
|
@ -3103,7 +3103,7 @@ public class HRegionServer extends HasThread implements
|
|||
protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
|
||||
throws NotServingRegionException {
|
||||
//Check for permissions to close.
|
||||
Region actualRegion = this.getFromOnlineRegions(encodedName);
|
||||
Region actualRegion = this.getRegion(encodedName);
|
||||
// Can be null if we're calling close on a region that's not online
|
||||
if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
|
||||
try {
|
||||
|
@ -3128,7 +3128,7 @@ public class HRegionServer extends HasThread implements
|
|||
return closeRegion(encodedName, abort, sn);
|
||||
}
|
||||
// Let's get the region from the online region list again
|
||||
actualRegion = this.getFromOnlineRegions(encodedName);
|
||||
actualRegion = this.getRegion(encodedName);
|
||||
if (actualRegion == null) { // If already online, we still need to close it.
|
||||
LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
|
||||
// The master deletes the znode when it receives this exception.
|
||||
|
@ -3170,7 +3170,7 @@ public class HRegionServer extends HasThread implements
|
|||
protected boolean closeAndOfflineRegionForSplitOrMerge(
|
||||
final List<String> regionEncodedName) throws IOException {
|
||||
for (int i = 0; i < regionEncodedName.size(); ++i) {
|
||||
Region regionToClose = this.getFromOnlineRegions(regionEncodedName.get(i));
|
||||
Region regionToClose = this.getRegion(regionEncodedName.get(i));
|
||||
if (regionToClose != null) {
|
||||
Map<byte[], List<HStoreFile>> hstoreFiles = null;
|
||||
Exception exceptionToThrow = null;
|
||||
|
@ -3211,7 +3211,7 @@ public class HRegionServer extends HasThread implements
|
|||
MetaTableAccessor.putToMetaTable(getConnection(), finalBarrier);
|
||||
}
|
||||
// Offline the region
|
||||
this.removeFromOnlineRegions(regionToClose, null);
|
||||
this.removeRegion(regionToClose, null);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
|
@ -3232,13 +3232,13 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public Region getFromOnlineRegions(final String encodedRegionName) {
|
||||
public Region getRegion(final String encodedRegionName) {
|
||||
return this.onlineRegions.get(encodedRegionName);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean removeFromOnlineRegions(final Region r, ServerName destination) {
|
||||
public boolean removeRegion(final Region r, ServerName destination) {
|
||||
Region toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
|
||||
if (destination != null) {
|
||||
long closeSeqNum = r.getMaxFlushedSeqId();
|
||||
|
|
|
@ -0,0 +1,60 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
/**
|
||||
* Interface to Map of online regions. In the Map, the key is the region's
|
||||
* encoded name and the value is an {@link Region} instance.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public interface ImmutableOnlineRegions {
|
||||
|
||||
/**
|
||||
* Return {@link Region} instance.
|
||||
* Only works if caller is in same context, in same JVM. Region is not
|
||||
* serializable.
|
||||
* @param encodedRegionName
|
||||
* @return Region for the passed encoded <code>encodedRegionName</code> or
|
||||
* null if named region is not member of the online regions.
|
||||
*/
|
||||
Region getRegion(String encodedRegionName);
|
||||
|
||||
/**
|
||||
* Get all online regions of a table in this RS.
|
||||
* @param tableName
|
||||
* @return List of Region
|
||||
* @throws java.io.IOException
|
||||
*/
|
||||
List<Region> getRegions(TableName tableName) throws IOException;
|
||||
|
||||
/**
|
||||
* Get all online regions in this RS.
|
||||
* @return List of online Region
|
||||
*/
|
||||
List<Region> getRegions();
|
||||
}
|
|
@ -110,7 +110,7 @@ public class IncreasingToUpperBoundRegionSplitPolicy extends ConstantSizeRegionS
|
|||
TableName tablename = region.getTableDescriptor().getTableName();
|
||||
int tableRegionsCount = 0;
|
||||
try {
|
||||
List<Region> hri = rss.getOnlineRegions(tablename);
|
||||
List<Region> hri = rss.getRegions(tablename);
|
||||
tableRegionsCount = hri == null || hri.isEmpty() ? 0 : hri.size();
|
||||
} catch (IOException e) {
|
||||
LOG.debug("Failed getOnlineRegions " + tablename, e);
|
||||
|
|
|
@ -204,7 +204,7 @@ public class LogRoller extends HasThread implements Closeable {
|
|||
*/
|
||||
private void scheduleFlush(final byte [] encodedRegionName) {
|
||||
boolean scheduled = false;
|
||||
Region r = this.services.getFromOnlineRegions(Bytes.toString(encodedRegionName));
|
||||
Region r = this.services.getRegion(Bytes.toString(encodedRegionName));
|
||||
FlushRequester requester = null;
|
||||
if (r != null) {
|
||||
requester = this.services.getFlushRequester();
|
||||
|
|
|
@ -18,28 +18,21 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Interface to Map of online regions. In the Map, the key is the region's
|
||||
* encoded name and the value is an {@link Region} instance.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public interface OnlineRegions extends Server {
|
||||
@InterfaceAudience.Private
|
||||
public interface OnlineRegions extends ImmutableOnlineRegions {
|
||||
|
||||
/**
|
||||
* Add to online regions.
|
||||
* @param r
|
||||
*/
|
||||
void addToOnlineRegions(final Region r);
|
||||
void addRegion(final Region r);
|
||||
|
||||
/**
|
||||
* This method removes Region corresponding to hri from the Map of onlineRegions.
|
||||
|
@ -48,29 +41,5 @@ public interface OnlineRegions extends Server {
|
|||
* @param destination Destination, if any, null otherwise.
|
||||
* @return True if we removed a region from online list.
|
||||
*/
|
||||
boolean removeFromOnlineRegions(final Region r, ServerName destination);
|
||||
|
||||
/**
|
||||
* Return {@link Region} instance.
|
||||
* Only works if caller is in same context, in same JVM. Region is not
|
||||
* serializable.
|
||||
* @param encodedRegionName
|
||||
* @return Region for the passed encoded <code>encodedRegionName</code> or
|
||||
* null if named region is not member of the online regions.
|
||||
*/
|
||||
Region getFromOnlineRegions(String encodedRegionName);
|
||||
|
||||
/**
|
||||
* Get all online regions of a table in this RS.
|
||||
* @param tableName
|
||||
* @return List of Region
|
||||
* @throws java.io.IOException
|
||||
*/
|
||||
List<Region> getOnlineRegions(TableName tableName) throws IOException;
|
||||
|
||||
/**
|
||||
* Get all online regions in this RS.
|
||||
* @return List of online Region
|
||||
*/
|
||||
List<Region> getOnlineRegions();
|
||||
boolean removeRegion(final Region r, ServerName destination);
|
||||
}
|
||||
|
|
|
@ -106,7 +106,7 @@ public class RSDumpServlet extends StateDumpServlet {
|
|||
|
||||
public static void dumpRowLock(HRegionServer hrs, PrintWriter out) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (Region region : hrs.getOnlineRegions()) {
|
||||
for (Region region : hrs.getRegions()) {
|
||||
HRegion hRegion = (HRegion)region;
|
||||
if (hRegion.getLockedRows().size() > 0) {
|
||||
for (HRegion.RowLockContext rowLockContext : hRegion.getLockedRows().values()) {
|
||||
|
|
|
@ -1712,9 +1712,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
List<Region> regions;
|
||||
if (request.hasTableName()) {
|
||||
TableName tableName = ProtobufUtil.toTableName(request.getTableName());
|
||||
regions = regionServer.getOnlineRegions(tableName);
|
||||
regions = regionServer.getRegions(tableName);
|
||||
} else {
|
||||
regions = regionServer.getOnlineRegions();
|
||||
regions = regionServer.getRegions();
|
||||
}
|
||||
List<RegionLoad> rLoads = new ArrayList<>(regions.size());
|
||||
RegionLoad.Builder regionLoadBuilder = ClusterStatusProtos.RegionLoad.newBuilder();
|
||||
|
@ -1902,7 +1902,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
try {
|
||||
String encodedName = region.getEncodedName();
|
||||
byte[] encodedNameBytes = region.getEncodedNameAsBytes();
|
||||
final Region onlineRegion = regionServer.getFromOnlineRegions(encodedName);
|
||||
final Region onlineRegion = regionServer.getRegion(encodedName);
|
||||
if (onlineRegion != null) {
|
||||
// The region is already online. This should not happen any more.
|
||||
String error = "Received OPEN for the region:"
|
||||
|
@ -1919,7 +1919,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
encodedNameBytes, Boolean.TRUE);
|
||||
|
||||
if (Boolean.FALSE.equals(previous)) {
|
||||
if (regionServer.getFromOnlineRegions(encodedName) != null) {
|
||||
if (regionServer.getRegion(encodedName) != null) {
|
||||
// There is a close in progress. This should not happen any more.
|
||||
String error = "Received OPEN for the region:"
|
||||
+ region.getRegionNameAsString() + ", which we are already trying to CLOSE";
|
||||
|
@ -2027,7 +2027,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
checkOpen();
|
||||
String encodedName = region.getEncodedName();
|
||||
byte[] encodedNameBytes = region.getEncodedNameAsBytes();
|
||||
final Region onlineRegion = regionServer.getFromOnlineRegions(encodedName);
|
||||
final Region onlineRegion = regionServer.getRegion(encodedName);
|
||||
|
||||
if (onlineRegion != null) {
|
||||
LOG.info("Region already online. Skipping warming up " + region);
|
||||
|
|
|
@ -138,7 +138,7 @@ public class RegionCoprocessorHost
|
|||
|
||||
/** @return reference to the region server services */
|
||||
@Override
|
||||
public RegionServerServices getRegionServerServices() {
|
||||
public CoprocessorRegionServerServices getCoprocessorRegionServerServices() {
|
||||
return rsServices;
|
||||
}
|
||||
|
||||
|
|
|
@ -309,7 +309,7 @@ public class RegionServerCoprocessorHost extends
|
|||
}
|
||||
|
||||
@Override
|
||||
public RegionServerServices getRegionServerServices() {
|
||||
public CoprocessorRegionServerServices getCoprocessorRegionServerServices() {
|
||||
return regionServerServices;
|
||||
}
|
||||
|
||||
|
|
|
@ -21,24 +21,21 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.client.locking.EntityLock;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||
import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
|
||||
import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
|
||||
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.protobuf.Service;
|
||||
|
@ -46,13 +43,9 @@ import com.google.protobuf.Service;
|
|||
/**
|
||||
* Services provided by {@link HRegionServer}
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegion {
|
||||
/**
|
||||
* @return True if this regionserver is stopping.
|
||||
*/
|
||||
boolean isStopping();
|
||||
@InterfaceAudience.Private
|
||||
public interface RegionServerServices
|
||||
extends Server, OnlineRegions, FavoredNodesForRegion, CoprocessorRegionServerServices {
|
||||
|
||||
/** @return the WAL for a particular region. Pass null for getting the
|
||||
* default (common) WAL */
|
||||
|
@ -188,11 +181,6 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
|
|||
*/
|
||||
ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS();
|
||||
|
||||
/**
|
||||
* @return Return the FileSystem object used by the regionserver
|
||||
*/
|
||||
FileSystem getFileSystem();
|
||||
|
||||
/**
|
||||
* @return The RegionServer's "Leases" service
|
||||
*/
|
||||
|
@ -214,12 +202,6 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
|
|||
*/
|
||||
public ServerNonceManager getNonceManager();
|
||||
|
||||
/**
|
||||
* @return all the online tables in this RS
|
||||
*/
|
||||
Set<TableName> getOnlineTables();
|
||||
|
||||
|
||||
/**
|
||||
* Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to be
|
||||
* available for handling
|
||||
|
|
|
@ -122,7 +122,7 @@ public class StorefileRefresherChore extends ScheduledChore {
|
|||
Iterator<String> lastRefreshTimesIter = lastRefreshTimes.keySet().iterator();
|
||||
while (lastRefreshTimesIter.hasNext()) {
|
||||
String encodedName = lastRefreshTimesIter.next();
|
||||
if (regionServer.getFromOnlineRegions(encodedName) == null) {
|
||||
if (regionServer.getRegion(encodedName) == null) {
|
||||
lastRefreshTimesIter.remove();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -90,7 +90,7 @@ public class CloseRegionHandler extends EventHandler {
|
|||
LOG.debug("Processing close of " + name);
|
||||
String encodedRegionName = regionInfo.getEncodedName();
|
||||
// Check that this region is being served here
|
||||
HRegion region = (HRegion)rsServices.getFromOnlineRegions(encodedRegionName);
|
||||
HRegion region = (HRegion)rsServices.getRegion(encodedRegionName);
|
||||
if (region == null) {
|
||||
LOG.warn("Received CLOSE for region " + name + " but currently not serving - ignoring");
|
||||
// TODO: do better than a simple warning
|
||||
|
@ -115,7 +115,7 @@ public class CloseRegionHandler extends EventHandler {
|
|||
throw new RuntimeException(ioe);
|
||||
}
|
||||
|
||||
this.rsServices.removeFromOnlineRegions(region, destination);
|
||||
this.rsServices.removeRegion(region, destination);
|
||||
rsServices.reportRegionStateTransition(TransitionCode.CLOSED, regionInfo);
|
||||
|
||||
// Done! Region is closed on this RS
|
||||
|
|
|
@ -88,7 +88,7 @@ public class OpenRegionHandler extends EventHandler {
|
|||
// 2) The region is now marked as online while we're suppose to open. This would be a bug.
|
||||
|
||||
// Check that this region is not already online
|
||||
if (this.rsServices.getFromOnlineRegions(encodedName) != null) {
|
||||
if (this.rsServices.getRegion(encodedName) != null) {
|
||||
LOG.error("Region " + encodedName +
|
||||
" was already online when we started processing the opening. " +
|
||||
"Marking this new attempt as failed");
|
||||
|
@ -119,7 +119,7 @@ public class OpenRegionHandler extends EventHandler {
|
|||
}
|
||||
|
||||
// Successful region open, and add it to OnlineRegions
|
||||
this.rsServices.addToOnlineRegions(region);
|
||||
this.rsServices.addRegion(region);
|
||||
openSuccessful = true;
|
||||
|
||||
// Done! Successful region open
|
||||
|
@ -313,7 +313,7 @@ public class OpenRegionHandler extends EventHandler {
|
|||
|
||||
void cleanupFailedOpen(final HRegion region) throws IOException {
|
||||
if (region != null) {
|
||||
this.rsServices.removeFromOnlineRegions(region, null);
|
||||
this.rsServices.removeRegion(region, null);
|
||||
region.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -223,7 +223,7 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
|
|||
* @throws IOException
|
||||
*/
|
||||
private List<Region> getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException {
|
||||
List<Region> onlineRegions = rss.getOnlineRegions(TableName.valueOf(snapshot.getTable()));
|
||||
List<Region> onlineRegions = rss.getRegions(TableName.valueOf(snapshot.getTable()));
|
||||
Iterator<Region> iterator = onlineRegions.iterator();
|
||||
// remove the non-default regions
|
||||
while (iterator.hasNext()) {
|
||||
|
|
|
@ -55,7 +55,7 @@ public class ReplicationObserver implements RegionObserver {
|
|||
+ "data replication.");
|
||||
return;
|
||||
}
|
||||
HRegionServer rs = (HRegionServer) env.getRegionServerServices();
|
||||
HRegionServer rs = (HRegionServer) env.getCoprocessorRegionServerServices();
|
||||
Replication rep = (Replication) rs.getReplicationSourceService();
|
||||
rep.addHFileRefsToQueue(env.getRegionInfo().getTable(), family, pairs);
|
||||
}
|
||||
|
|
|
@ -104,6 +104,7 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
|||
import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
|
@ -949,12 +950,14 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
zk = mEnv.getMasterServices().getZooKeeper();
|
||||
} else if (env instanceof RegionServerCoprocessorEnvironment) {
|
||||
RegionServerCoprocessorEnvironment rsEnv = (RegionServerCoprocessorEnvironment) env;
|
||||
zk = rsEnv.getRegionServerServices().getZooKeeper();
|
||||
assert rsEnv.getCoprocessorRegionServerServices() instanceof RegionServerServices;
|
||||
zk = ((RegionServerServices) rsEnv.getCoprocessorRegionServerServices()).getZooKeeper();
|
||||
} else if (env instanceof RegionCoprocessorEnvironment) {
|
||||
// if running at region
|
||||
regionEnv = (RegionCoprocessorEnvironment) env;
|
||||
conf.addBytesMap(regionEnv.getRegion().getTableDescriptor().getValues());
|
||||
zk = regionEnv.getRegionServerServices().getZooKeeper();
|
||||
assert regionEnv.getCoprocessorRegionServerServices() instanceof RegionServerServices;
|
||||
zk = ((RegionServerServices) regionEnv.getCoprocessorRegionServerServices()).getZooKeeper();
|
||||
compatibleEarlyTermination = conf.getBoolean(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT,
|
||||
AccessControlConstants.DEFAULT_ATTRIBUTE_EARLY_OUT);
|
||||
}
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
|||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.security.AccessDeniedException;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
@ -61,7 +62,9 @@ public class TokenProvider implements AuthenticationProtos.AuthenticationService
|
|||
if (env instanceof RegionCoprocessorEnvironment) {
|
||||
RegionCoprocessorEnvironment regionEnv =
|
||||
(RegionCoprocessorEnvironment)env;
|
||||
RpcServerInterface server = regionEnv.getRegionServerServices().getRpcServer();
|
||||
assert regionEnv.getCoprocessorRegionServerServices() instanceof RegionServerServices;
|
||||
RpcServerInterface server = ((RegionServerServices) regionEnv
|
||||
.getCoprocessorRegionServerServices()).getRpcServer();
|
||||
SecretManager<?> mgr = ((RpcServer)server).getSecretManager();
|
||||
if (mgr instanceof AuthenticationTokenSecretManager) {
|
||||
secretManager = (AuthenticationTokenSecretManager)mgr;
|
||||
|
|
|
@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.io.util.StreamUtils;
|
|||
import org.apache.hadoop.hbase.regionserver.OperationStatus;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.security.Superusers;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -111,7 +112,9 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
|
|||
|
||||
@Override
|
||||
public void init(RegionCoprocessorEnvironment e) throws IOException {
|
||||
ZooKeeperWatcher zk = e.getRegionServerServices().getZooKeeper();
|
||||
assert e.getCoprocessorRegionServerServices() instanceof RegionServerServices;
|
||||
ZooKeeperWatcher zk = ((RegionServerServices) e.getCoprocessorRegionServerServices())
|
||||
.getZooKeeper();
|
||||
try {
|
||||
labelsCache = VisibilityLabelsCache.createAndGet(zk, this.conf);
|
||||
} catch (IOException ioe) {
|
||||
|
|
|
@ -36,7 +36,7 @@
|
|||
HRegionServer rs = (HRegionServer) getServletContext().getAttribute(HRegionServer.REGIONSERVER);
|
||||
Configuration conf = rs.getConfiguration();
|
||||
|
||||
Region region = rs.getFromOnlineRegions(regionName);
|
||||
Region region = rs.getRegion(regionName);
|
||||
String displayName = HRegionInfo.getRegionNameAsStringForDisplay(region.getRegionInfo(),
|
||||
rs.getConfiguration());
|
||||
%>
|
||||
|
|
|
@ -4442,7 +4442,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
public int getNumHFilesForRS(final HRegionServer rs, final TableName tableName,
|
||||
final byte[] family) {
|
||||
int numHFiles = 0;
|
||||
for (Region region : rs.getOnlineRegions(tableName)) {
|
||||
for (Region region : rs.getRegions(tableName)) {
|
||||
numHFiles += region.getStore(family).getStorefilesCount();
|
||||
}
|
||||
return numHFiles;
|
||||
|
|
|
@ -801,7 +801,7 @@ public class MiniHBaseCluster extends HBaseCluster {
|
|||
ArrayList<HRegion> ret = new ArrayList<>();
|
||||
for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
|
||||
HRegionServer hrs = rst.getRegionServer();
|
||||
for (Region region : hrs.getOnlineRegions(tableName)) {
|
||||
for (Region region : hrs.getRegions(tableName)) {
|
||||
if (region.getTableDescriptor().getTableName().equals(tableName)) {
|
||||
ret.add((HRegion)region);
|
||||
}
|
||||
|
|
|
@ -94,17 +94,17 @@ public class MockRegionServerServices implements RegionServerServices {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean removeFromOnlineRegions(Region r, ServerName destination) {
|
||||
public boolean removeRegion(Region r, ServerName destination) {
|
||||
return this.regions.remove(r.getRegionInfo().getEncodedName()) != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Region getFromOnlineRegions(String encodedRegionName) {
|
||||
public Region getRegion(String encodedRegionName) {
|
||||
return this.regions.get(encodedRegionName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Region> getOnlineRegions(TableName tableName) throws IOException {
|
||||
public List<Region> getRegions(TableName tableName) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -114,24 +114,24 @@ public class MockRegionServerServices implements RegionServerServices {
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<Region> getOnlineRegions() {
|
||||
public List<Region> getRegions() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addToOnlineRegions(Region r) {
|
||||
public void addRegion(Region r) {
|
||||
this.regions.put(r.getRegionInfo().getEncodedName(), r);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postOpenDeployTasks(Region r) throws KeeperException, IOException {
|
||||
addToOnlineRegions(r);
|
||||
addRegion(r);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postOpenDeployTasks(PostOpenDeployContext context) throws KeeperException,
|
||||
IOException {
|
||||
addToOnlineRegions(context.getRegion());
|
||||
addRegion(context.getRegion());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -93,9 +93,7 @@ public class TestGlobalMemStoreSize {
|
|||
long globalMemStoreSize = 0;
|
||||
for (HRegionInfo regionInfo :
|
||||
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
|
||||
globalMemStoreSize +=
|
||||
server.getFromOnlineRegions(regionInfo.getEncodedName()).
|
||||
getMemstoreSize();
|
||||
globalMemStoreSize += server.getRegion(regionInfo.getEncodedName()).getMemstoreSize();
|
||||
}
|
||||
assertEquals(server.getRegionServerAccounting().getGlobalMemstoreDataSize(),
|
||||
globalMemStoreSize);
|
||||
|
@ -109,7 +107,7 @@ public class TestGlobalMemStoreSize {
|
|||
|
||||
for (HRegionInfo regionInfo :
|
||||
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
|
||||
Region r = server.getFromOnlineRegions(regionInfo.getEncodedName());
|
||||
Region r = server.getRegion(regionInfo.getEncodedName());
|
||||
flush(r, server);
|
||||
}
|
||||
LOG.info("Post flush on " + server.getServerName());
|
||||
|
@ -125,7 +123,7 @@ public class TestGlobalMemStoreSize {
|
|||
// our test was running....
|
||||
for (HRegionInfo regionInfo :
|
||||
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
|
||||
Region r = server.getFromOnlineRegions(regionInfo.getEncodedName());
|
||||
Region r = server.getRegion(regionInfo.getEncodedName());
|
||||
long l = r.getMemstoreSize();
|
||||
if (l > 0) {
|
||||
// Only meta could have edits at this stage. Give it another flush
|
||||
|
|
|
@ -241,7 +241,7 @@ public class TestHFileArchiving {
|
|||
UTIL.loadRegion(region, TEST_FAM);
|
||||
|
||||
// get the hfiles in the region
|
||||
List<Region> regions = hrs.getOnlineRegions(tableName);
|
||||
List<Region> regions = hrs.getRegions(tableName);
|
||||
assertEquals("More that 1 region for test table.", 1, regions.size());
|
||||
|
||||
region = regions.get(0);
|
||||
|
@ -320,7 +320,7 @@ public class TestHFileArchiving {
|
|||
UTIL.loadRegion(region, TEST_FAM);
|
||||
|
||||
// get the hfiles in the region
|
||||
List<Region> regions = hrs.getOnlineRegions(tableName);
|
||||
List<Region> regions = hrs.getRegions(tableName);
|
||||
assertEquals("More that 1 region for test table.", 1, regions.size());
|
||||
|
||||
region = regions.get(0);
|
||||
|
|
|
@ -180,7 +180,7 @@ public class TestZooKeeperTableArchiveClient {
|
|||
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
|
||||
List<Region> regions = new ArrayList<>();
|
||||
regions.add(region);
|
||||
when(rss.getOnlineRegions()).thenReturn(regions);
|
||||
when(rss.getRegions()).thenReturn(regions);
|
||||
final CompactedHFilesDischarger compactionCleaner =
|
||||
new CompactedHFilesDischarger(100, stop, rss, false);
|
||||
loadFlushAndCompact(region, TEST_FAM);
|
||||
|
@ -233,7 +233,7 @@ public class TestZooKeeperTableArchiveClient {
|
|||
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
|
||||
List<Region> regions = new ArrayList<>();
|
||||
regions.add(region);
|
||||
when(rss.getOnlineRegions()).thenReturn(regions);
|
||||
when(rss.getRegions()).thenReturn(regions);
|
||||
final CompactedHFilesDischarger compactionCleaner =
|
||||
new CompactedHFilesDischarger(100, stop, rss, false);
|
||||
loadFlushAndCompact(region, TEST_FAM);
|
||||
|
@ -243,7 +243,7 @@ public class TestZooKeeperTableArchiveClient {
|
|||
HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd);
|
||||
regions = new ArrayList<>();
|
||||
regions.add(otherRegion);
|
||||
when(rss.getOnlineRegions()).thenReturn(regions);
|
||||
when(rss.getRegions()).thenReturn(regions);
|
||||
final CompactedHFilesDischarger compactionCleaner1 = new CompactedHFilesDischarger(100, stop,
|
||||
rss, false);
|
||||
loadFlushAndCompact(otherRegion, TEST_FAM);
|
||||
|
|
|
@ -64,7 +64,7 @@ public class TestAsyncMetaRegionLocator {
|
|||
private Optional<ServerName> getRSCarryingMeta() {
|
||||
return TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
|
||||
.map(t -> t.getRegionServer())
|
||||
.filter(rs -> !rs.getOnlineRegions(TableName.META_TABLE_NAME).isEmpty()).findAny()
|
||||
.filter(rs -> !rs.getRegions(TableName.META_TABLE_NAME).isEmpty()).findAny()
|
||||
.map(rs -> rs.getServerName());
|
||||
}
|
||||
|
||||
|
|
|
@ -176,7 +176,7 @@ public class TestAsyncNonMetaRegionLocator {
|
|||
ServerName[] serverNames = new ServerName[startKeys.length];
|
||||
TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
|
||||
.forEach(rs -> {
|
||||
rs.getOnlineRegions(TABLE_NAME).forEach(r -> {
|
||||
rs.getRegions(TABLE_NAME).forEach(r -> {
|
||||
serverNames[Arrays.binarySearch(startKeys, r.getRegionInfo().getStartKey(),
|
||||
Bytes::compareTo)] = rs.getServerName();
|
||||
});
|
||||
|
@ -270,7 +270,7 @@ public class TestAsyncNonMetaRegionLocator {
|
|||
LOCATOR.getRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER, false).get();
|
||||
ServerName afterServerName =
|
||||
TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
|
||||
.filter(rs -> rs.getOnlineRegions(TABLE_NAME).stream()
|
||||
.filter(rs -> rs.getRegions(TABLE_NAME).stream()
|
||||
.anyMatch(r -> Bytes.equals(splitKey, r.getRegionInfo().getStartKey())))
|
||||
.findAny().get().getServerName();
|
||||
assertLocEquals(splitKey, EMPTY_END_ROW, afterServerName, afterLoc);
|
||||
|
|
|
@ -301,7 +301,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
ServerName serverName = rs.getServerName();
|
||||
try {
|
||||
Assert.assertEquals(admin.getOnlineRegions(serverName).get().size(), rs
|
||||
.getOnlineRegions().size());
|
||||
.getRegions().size());
|
||||
} catch (Exception e) {
|
||||
fail("admin.getOnlineRegions() method throws a exception: " + e.getMessage());
|
||||
}
|
||||
|
@ -527,7 +527,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().stream()
|
||||
.map(rsThread -> rsThread.getRegionServer()).collect(Collectors.toList());
|
||||
List<Region> regions = new ArrayList<>();
|
||||
rsList.forEach(rs -> regions.addAll(rs.getOnlineRegions(tableName)));
|
||||
rsList.forEach(rs -> regions.addAll(rs.getRegions(tableName)));
|
||||
Assert.assertEquals(regions.size(), 1);
|
||||
int countBefore = countStoreFilesInFamilies(regions, families);
|
||||
Assert.assertTrue(countBefore > 0);
|
||||
|
@ -568,7 +568,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
|||
TEST_UTIL
|
||||
.getHBaseCluster()
|
||||
.getLiveRegionServerThreads()
|
||||
.forEach(rsThread -> regions.addAll(rsThread.getRegionServer().getOnlineRegions(tableName)));
|
||||
.forEach(rsThread -> regions.addAll(rsThread.getRegionServer().getRegions(tableName)));
|
||||
Assert.assertEquals(regions.size(), 1);
|
||||
|
||||
int countBefore = countStoreFilesInFamilies(regions, families);
|
||||
|
|
|
@ -133,7 +133,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
|
|||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region =
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
@ -307,7 +307,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
|
|||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region =
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
|
|
@ -182,8 +182,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
@ -273,8 +272,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
@ -332,8 +330,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
@ -394,8 +391,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
BlockCache cache = setCacheProperties(region);
|
||||
Put put = new Put(ROW);
|
||||
put.addColumn(FAMILY, QUALIFIER, data);
|
||||
|
@ -487,8 +483,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
BlockCache cache = setCacheProperties(region);
|
||||
|
||||
Put put = new Put(ROW);
|
||||
|
@ -571,8 +566,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region =
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -630,8 +624,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
@ -713,8 +706,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
BlockCache cache = setCacheProperties(region);
|
||||
|
||||
Put put = new Put(ROW);
|
||||
|
@ -813,8 +805,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
@ -879,8 +870,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
@ -997,8 +987,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
@ -1127,8 +1116,7 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
|
||||
regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
|
|
@ -96,7 +96,7 @@ public class TestClientPushback {
|
|||
BufferedMutatorImpl mutator = (BufferedMutatorImpl) conn.getBufferedMutator(tableName);
|
||||
|
||||
HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0);
|
||||
Region region = rs.getOnlineRegions(tableName).get(0);
|
||||
Region region = rs.getRegions(tableName).get(0);
|
||||
|
||||
LOG.debug("Writing some data to "+tableName);
|
||||
// write some data
|
||||
|
@ -183,7 +183,7 @@ public class TestClientPushback {
|
|||
ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
|
||||
Table table = conn.getTable(tableName);
|
||||
HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0);
|
||||
Region region = rs.getOnlineRegions(tableName).get(0);
|
||||
Region region = rs.getRegions(tableName).get(0);
|
||||
|
||||
RowMutations mutations = new RowMutations(Bytes.toBytes("row"));
|
||||
Put p = new Put(Bytes.toBytes("row"));
|
||||
|
|
|
@ -4528,8 +4528,7 @@ public class TestFromClientSide {
|
|||
// set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow
|
||||
// in Store.rowAtOrBeforeFromStoreFile
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region =
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Put put1 = new Put(firstRow);
|
||||
Put put2 = new Put(secondRow);
|
||||
Put put3 = new Put(thirdRow);
|
||||
|
@ -5295,8 +5294,7 @@ public class TestFromClientSide {
|
|||
// get the block cache and region
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName)
|
||||
.getFromOnlineRegions(regionName);
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
|
|
|
@ -961,7 +961,7 @@ public class TestFromClientSide3 {
|
|||
private static Region find(final TableName tableName)
|
||||
throws IOException, InterruptedException {
|
||||
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
|
||||
List<Region> regions = rs.getOnlineRegions(tableName);
|
||||
List<Region> regions = rs.getRegions(tableName);
|
||||
assertEquals(1, regions.size());
|
||||
return regions.get(0);
|
||||
}
|
||||
|
|
|
@ -1328,7 +1328,7 @@ public class TestHCM {
|
|||
assertTrue(!destServerName.equals(metaServerName));
|
||||
|
||||
//find another row in the cur server that is less than ROW_X
|
||||
List<Region> regions = curServer.getOnlineRegions(TABLE_NAME3);
|
||||
List<Region> regions = curServer.getRegions(TABLE_NAME3);
|
||||
byte[] otherRow = null;
|
||||
for (Region region : regions) {
|
||||
if (!region.getRegionInfo().getEncodedName().equals(toMove.getRegionInfo().getEncodedName())
|
||||
|
|
|
@ -159,7 +159,7 @@ public class TestMultiRespectsLimits {
|
|||
TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
|
||||
@Override
|
||||
public boolean evaluate() throws Exception {
|
||||
return regionServer.getOnlineRegions(tableName).get(0).getMaxFlushedSeqId() > 3;
|
||||
return regionServer.getRegions(tableName).get(0).getMaxFlushedSeqId() > 3;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -134,7 +134,7 @@ public class TestReplicaWithCluster {
|
|||
if (e.getEnvironment().getRegion().getRegionInfo().getReplicaId() <= 1) {
|
||||
LOG.info("Throw Region Server Stopped Exceptoin for replica id " + replicaId);
|
||||
throw new RegionServerStoppedException("Server " +
|
||||
e.getEnvironment().getRegionServerServices().getServerName()
|
||||
e.getEnvironment().getCoprocessorRegionServerServices().getServerName()
|
||||
+ " not running");
|
||||
} else {
|
||||
LOG.info("We're replica region " + replicaId);
|
||||
|
@ -151,7 +151,7 @@ public class TestReplicaWithCluster {
|
|||
if (e.getEnvironment().getRegion().getRegionInfo().getReplicaId() <= 1) {
|
||||
LOG.info("Throw Region Server Stopped Exceptoin for replica id " + replicaId);
|
||||
throw new RegionServerStoppedException("Server " +
|
||||
e.getEnvironment().getRegionServerServices().getServerName()
|
||||
e.getEnvironment().getCoprocessorRegionServerServices().getServerName()
|
||||
+ " not running");
|
||||
} else {
|
||||
LOG.info("We're replica region " + replicaId);
|
||||
|
@ -179,8 +179,9 @@ public class TestReplicaWithCluster {
|
|||
if (!e.getEnvironment().getRegion().getRegionInfo().isMetaRegion() && (replicaId == 0)) {
|
||||
LOG.info("Get, throw Region Server Stopped Exceptoin for region " + e.getEnvironment()
|
||||
.getRegion().getRegionInfo());
|
||||
throw new RegionServerStoppedException("Server " +
|
||||
e.getEnvironment().getRegionServerServices().getServerName() + " not running");
|
||||
throw new RegionServerStoppedException(
|
||||
"Server " + e.getEnvironment().getCoprocessorRegionServerServices().getServerName()
|
||||
+ " not running");
|
||||
}
|
||||
} else {
|
||||
LOG.info("Get, We're replica region " + replicaId);
|
||||
|
@ -209,8 +210,9 @@ public class TestReplicaWithCluster {
|
|||
LOG.info("Scan, throw Region Server Stopped Exceptoin for replica " + e.getEnvironment()
|
||||
.getRegion().getRegionInfo());
|
||||
|
||||
throw new RegionServerStoppedException("Server " +
|
||||
e.getEnvironment().getRegionServerServices().getServerName() + " not running");
|
||||
throw new RegionServerStoppedException(
|
||||
"Server " + e.getEnvironment().getCoprocessorRegionServerServices().getServerName()
|
||||
+ " not running");
|
||||
} else {
|
||||
LOG.info("Scan, We're replica region " + replicaId);
|
||||
}
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.io.Reference;
|
|||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.Leases;
|
||||
import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
|
||||
import org.apache.hadoop.hbase.regionserver.Region.Operation;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
|
@ -137,12 +136,6 @@ public class SimpleRegionObserver implements RegionObserver {
|
|||
|
||||
@Override
|
||||
public void start(CoprocessorEnvironment e) throws IOException {
|
||||
// this only makes sure that leases and locks are available to coprocessors
|
||||
// from external packages
|
||||
RegionCoprocessorEnvironment re = (RegionCoprocessorEnvironment)e;
|
||||
Leases leases = re.getRegionServerServices().getLeases();
|
||||
leases.createLease(re.getRegion().getRegionInfo().getRegionNameAsString(), 2000, null);
|
||||
leases.cancelLease(re.getRegion().getRegionInfo().getRegionNameAsString());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -300,7 +300,7 @@ public class TestRegionObserverScannerOpenHook {
|
|||
table.put(put);
|
||||
|
||||
HRegionServer rs = UTIL.getRSForFirstRegionInTable(desc.getTableName());
|
||||
List<Region> regions = rs.getOnlineRegions(desc.getTableName());
|
||||
List<Region> regions = rs.getRegions(desc.getTableName());
|
||||
assertEquals("More than 1 region serving test table with 1 row", 1, regions.size());
|
||||
Region region = regions.get(0);
|
||||
admin.flushRegion(region.getRegionInfo().getRegionName());
|
||||
|
|
|
@ -292,7 +292,7 @@ public class TestBlockReorder {
|
|||
|
||||
int nbTest = 0;
|
||||
while (nbTest < 10) {
|
||||
final List<Region> regions = targetRs.getOnlineRegions(h.getName());
|
||||
final List<Region> regions = targetRs.getRegions(h.getName());
|
||||
final CountDownLatch latch = new CountDownLatch(regions.size());
|
||||
// listen for successful log rolls
|
||||
final WALActionsListener listener = new WALActionsListener.Base() {
|
||||
|
|
|
@ -263,18 +263,18 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void addToOnlineRegions(Region r) {
|
||||
public void addRegion(Region r) {
|
||||
// TODO Auto-generated method stub
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean removeFromOnlineRegions(Region r, ServerName destination) {
|
||||
public boolean removeRegion(Region r, ServerName destination) {
|
||||
// TODO Auto-generated method stub
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HRegion getFromOnlineRegions(String encodedRegionName) {
|
||||
public HRegion getRegion(String encodedRegionName) {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
|
@ -468,7 +468,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<Region> getOnlineRegions() {
|
||||
public List<Region> getRegions() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -535,7 +535,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<Region> getOnlineRegions(TableName tableName) throws IOException {
|
||||
public List<Region> getRegions(TableName tableName) throws IOException {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -258,7 +258,7 @@ public class TestAssignmentListener {
|
|||
admin.majorCompact(tableName);
|
||||
mergeable = 0;
|
||||
for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) {
|
||||
for (Region region: regionThread.getRegionServer().getOnlineRegions(tableName)) {
|
||||
for (Region region: regionThread.getRegionServer().getRegions(tableName)) {
|
||||
mergeable += ((HRegion)region).isMergeable() ? 1 : 0;
|
||||
}
|
||||
}
|
||||
|
@ -299,7 +299,7 @@ public class TestAssignmentListener {
|
|||
MiniHBaseCluster miniCluster = TEST_UTIL.getMiniHBaseCluster();
|
||||
int serverCount = 0;
|
||||
for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) {
|
||||
if (!regionThread.getRegionServer().getOnlineRegions(TABLE_NAME).isEmpty()) {
|
||||
if (!regionThread.getRegionServer().getRegions(TABLE_NAME).isEmpty()) {
|
||||
++serverCount;
|
||||
}
|
||||
if (serverCount > 1) {
|
||||
|
|
|
@ -80,7 +80,7 @@ public class TestGetLastFlushedSequenceId {
|
|||
Region region = null;
|
||||
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
||||
HRegionServer hrs = rsts.get(i).getRegionServer();
|
||||
for (Region r : hrs.getOnlineRegions(tableName)) {
|
||||
for (Region r : hrs.getRegions(tableName)) {
|
||||
region = r;
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -225,8 +225,8 @@ public class TestMasterFailover {
|
|||
// region server should expire (how it can be verified?)
|
||||
MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(),
|
||||
rs.getServerName(), State.OPENING);
|
||||
Region meta = rs.getFromOnlineRegions(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
|
||||
rs.removeFromOnlineRegions(meta, null);
|
||||
Region meta = rs.getRegion(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
|
||||
rs.removeRegion(meta, null);
|
||||
((HRegion)meta).close();
|
||||
|
||||
log("Aborting master");
|
||||
|
|
|
@ -417,7 +417,7 @@ public class TestRegionPlacement {
|
|||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
for (int i = 0; i < SLAVES; i++) {
|
||||
HRegionServer rs = cluster.getRegionServer(i);
|
||||
for (Region region: rs.getOnlineRegions(TableName.valueOf("testRegionAssignment"))) {
|
||||
for (Region region: rs.getRegions(TableName.valueOf("testRegionAssignment"))) {
|
||||
InetSocketAddress[] favoredSocketAddress = rs.getFavoredNodesForRegion(
|
||||
region.getRegionInfo().getEncodedName());
|
||||
List<ServerName> favoredServerList = plan.getAssignmentMap().get(region.getRegionInfo());
|
||||
|
|
|
@ -544,7 +544,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
|
|||
|
||||
private void compactTable(TableName tableName) throws IOException {
|
||||
for(JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
|
||||
for(Region region : t.getRegionServer().getOnlineRegions(tableName)) {
|
||||
for(Region region : t.getRegionServer().getRegions(tableName)) {
|
||||
region.compact(true);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -62,7 +62,7 @@ public class TestRegionLocationFinder {
|
|||
|
||||
for (int i = 0; i < ServerNum; i++) {
|
||||
HRegionServer server = cluster.getRegionServer(i);
|
||||
for (Region region : server.getOnlineRegions(tableName)) {
|
||||
for (Region region : server.getRegions(tableName)) {
|
||||
region.flush(true);
|
||||
}
|
||||
}
|
||||
|
@ -83,7 +83,7 @@ public class TestRegionLocationFinder {
|
|||
public void testInternalGetTopBlockLocation() throws Exception {
|
||||
for (int i = 0; i < ServerNum; i++) {
|
||||
HRegionServer server = cluster.getRegionServer(i);
|
||||
for (Region region : server.getOnlineRegions(tableName)) {
|
||||
for (Region region : server.getRegions(tableName)) {
|
||||
// get region's hdfs block distribution by region and RegionLocationFinder,
|
||||
// they should have same result
|
||||
HDFSBlocksDistribution blocksDistribution1 = region.getHDFSBlocksDistribution();
|
||||
|
@ -122,7 +122,7 @@ public class TestRegionLocationFinder {
|
|||
public void testGetTopBlockLocations() throws Exception {
|
||||
for (int i = 0; i < ServerNum; i++) {
|
||||
HRegionServer server = cluster.getRegionServer(i);
|
||||
for (Region region : server.getOnlineRegions(tableName)) {
|
||||
for (Region region : server.getRegions(tableName)) {
|
||||
List<ServerName> servers = finder.getTopBlockLocations(region
|
||||
.getRegionInfo());
|
||||
// test table may have empty region
|
||||
|
@ -147,7 +147,7 @@ public class TestRegionLocationFinder {
|
|||
finder.getCache().invalidateAll();
|
||||
for (int i = 0; i < ServerNum; i++) {
|
||||
HRegionServer server = cluster.getRegionServer(i);
|
||||
List<Region> regions = server.getOnlineRegions(tableName);
|
||||
List<Region> regions = server.getRegions(tableName);
|
||||
if (regions.size() <= 0) {
|
||||
continue;
|
||||
}
|
||||
|
|
|
@ -150,7 +150,7 @@ public class TestRegionsOnMasterOptions {
|
|||
try {
|
||||
Table t = TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY, REGIONS);
|
||||
LOG.info("Server: " + cluster.getMaster().getServerManager().getOnlineServersList());
|
||||
List<Region> regions = cluster.getMaster().getOnlineRegions();
|
||||
List<Region> regions = cluster.getMaster().getRegions();
|
||||
int mActualCount = regions.size();
|
||||
if (masterCount == 0 || masterCount == SYSTEM_REGIONS) {
|
||||
// 0 means no regions on master.
|
||||
|
@ -163,7 +163,7 @@ public class TestRegionsOnMasterOptions {
|
|||
// thread though it is a regionserver so we have to check master and then below the
|
||||
// regionservers.
|
||||
for (JVMClusterUtil.RegionServerThread rst: cluster.getRegionServerThreads()) {
|
||||
regions = rst.getRegionServer().getOnlineRegions();
|
||||
regions = rst.getRegionServer().getRegions();
|
||||
int rsActualCount = regions.size();
|
||||
checkCount(rsActualCount, rsCount);
|
||||
}
|
||||
|
@ -179,7 +179,7 @@ public class TestRegionsOnMasterOptions {
|
|||
}
|
||||
LOG.info("Cluster is up; running balancer");
|
||||
cluster.getMaster().balance();
|
||||
regions = cluster.getMaster().getOnlineRegions();
|
||||
regions = cluster.getMaster().getRegions();
|
||||
int mNewActualCount = regions.size();
|
||||
if (masterCount == 0 || masterCount == SYSTEM_REGIONS) {
|
||||
// 0 means no regions on master. After crash, should still be no regions on master.
|
||||
|
|
|
@ -328,7 +328,7 @@ public class TestSnapshotFromMaster {
|
|||
.getRegionServerThreads();
|
||||
HRegionServer hrs = null;
|
||||
for (RegionServerThread rs : regionServerThreads) {
|
||||
if (!rs.getRegionServer().getOnlineRegions(TABLE_NAME).isEmpty()) {
|
||||
if (!rs.getRegionServer().getRegions(TABLE_NAME).isEmpty()) {
|
||||
hrs = rs.getRegionServer();
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -64,7 +64,7 @@ public class TestFileSystemUtilizationChore {
|
|||
.reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
|
||||
|
||||
final Region region = mockRegionWithSize(regionSizes);
|
||||
when(rs.getOnlineRegions()).thenReturn(Arrays.asList(region));
|
||||
when(rs.getRegions()).thenReturn(Arrays.asList(region));
|
||||
chore.chore();
|
||||
}
|
||||
|
||||
|
@ -81,7 +81,7 @@ public class TestFileSystemUtilizationChore {
|
|||
.reportRegionSizesForQuotas((Map<HRegionInfo,Long>) any(Map.class));
|
||||
|
||||
final Region region = mockRegionWithSize(regionSizes);
|
||||
when(rs.getOnlineRegions()).thenReturn(Arrays.asList(region));
|
||||
when(rs.getRegions()).thenReturn(Arrays.asList(region));
|
||||
chore.chore();
|
||||
}
|
||||
|
||||
|
@ -107,7 +107,7 @@ public class TestFileSystemUtilizationChore {
|
|||
final Region r1 = mockRegionWithSize(r1Sizes);
|
||||
final Region r2 = mockRegionWithSize(r2Sizes);
|
||||
final Region r3 = mockRegionWithSize(r3Sizes);
|
||||
when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2, r3));
|
||||
when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2, r3));
|
||||
chore.chore();
|
||||
}
|
||||
|
||||
|
@ -173,7 +173,7 @@ public class TestFileSystemUtilizationChore {
|
|||
final Region r1 = mockRegionWithSize(Arrays.asList(1024L, 2048L));
|
||||
final Region r2 = mockRegionWithSize(Arrays.asList(1024L * 1024L));
|
||||
final Region r3 = mockRegionWithSize(Arrays.asList(10L * 1024L * 1024L));
|
||||
when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2, r3, lr1, lr2));
|
||||
when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2, r3, lr1, lr2));
|
||||
|
||||
chore.chore();
|
||||
}
|
||||
|
@ -206,7 +206,7 @@ public class TestFileSystemUtilizationChore {
|
|||
final Region r2 = mockRegionWithSize(Arrays.asList(1024L * 1024L));
|
||||
final Region r3 = mockRegionWithSize(Arrays.asList(10L * 1024L * 1024L));
|
||||
// lr2 is no longer online, so it should be ignored
|
||||
when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2, r3, lr1));
|
||||
when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2, r3, lr1));
|
||||
|
||||
chore.chore();
|
||||
}
|
||||
|
@ -229,7 +229,7 @@ public class TestFileSystemUtilizationChore {
|
|||
|
||||
final Region r1 = mockRegionWithSize(r1Sizes);
|
||||
final Region r2 = mockSplitParentRegionWithSize(r2Sizes);
|
||||
when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2));
|
||||
when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2));
|
||||
chore.chore();
|
||||
}
|
||||
|
||||
|
@ -251,7 +251,7 @@ public class TestFileSystemUtilizationChore {
|
|||
|
||||
final Region r1 = mockRegionWithSize(r1Sizes);
|
||||
final Region r2 = mockRegionReplicaWithSize(r2Sizes);
|
||||
when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2));
|
||||
when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2));
|
||||
chore.chore();
|
||||
}
|
||||
|
||||
|
@ -278,7 +278,7 @@ public class TestFileSystemUtilizationChore {
|
|||
|
||||
final Region r1 = mockRegionWithHFileLinks(r1StoreFileSizes, r1HFileSizes);
|
||||
final Region r2 = mockRegionWithHFileLinks(r2StoreFileSizes, r2HFileSizes);
|
||||
when(rs.getOnlineRegions()).thenReturn(Arrays.asList(r1, r2));
|
||||
when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2));
|
||||
chore.chore();
|
||||
}
|
||||
|
||||
|
|
|
@ -92,7 +92,7 @@ public class TestCompactionArchiveConcurrentClose {
|
|||
RegionServerServices rss = mock(RegionServerServices.class);
|
||||
List<Region> regions = new ArrayList<>();
|
||||
regions.add(region);
|
||||
when(rss.getOnlineRegions()).thenReturn(regions);
|
||||
when(rss.getRegions()).thenReturn(regions);
|
||||
|
||||
// Create the cleaner object
|
||||
CompactedHFilesDischarger cleaner =
|
||||
|
|
|
@ -101,7 +101,7 @@ public class TestCompactionArchiveIOException {
|
|||
RegionServerServices rss = mock(RegionServerServices.class);
|
||||
List<Region> regions = new ArrayList<>();
|
||||
regions.add(region);
|
||||
when(rss.getOnlineRegions()).thenReturn(regions);
|
||||
when(rss.getRegions()).thenReturn(regions);
|
||||
|
||||
// Create the cleaner object
|
||||
final CompactedHFilesDischarger cleaner =
|
||||
|
|
|
@ -133,7 +133,7 @@ public class TestCompactionFileNotFound {
|
|||
int numRegionsAfterSplit = 0;
|
||||
List<RegionServerThread> rst = util.getMiniHBaseCluster().getLiveRegionServerThreads();
|
||||
for (RegionServerThread t : rst) {
|
||||
numRegionsAfterSplit += t.getRegionServer().getOnlineRegions(TEST_TABLE).size();
|
||||
numRegionsAfterSplit += t.getRegionServer().getRegions(TEST_TABLE).size();
|
||||
}
|
||||
// Make sure that the split went through and all the regions are assigned
|
||||
return (numRegionsAfterSplit == numRegionsBeforeSplit + 1
|
||||
|
|
|
@ -125,7 +125,7 @@ public class TestCompactionInDeadRegionServer {
|
|||
@Test
|
||||
public void test() throws Exception {
|
||||
HRegionServer rsToSuspend = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
|
||||
HRegion region = (HRegion) rsToSuspend.getOnlineRegions(TABLE_NAME).get(0);
|
||||
HRegion region = (HRegion) rsToSuspend.getRegions(TABLE_NAME).get(0);
|
||||
ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher();
|
||||
watcher.getRecoverableZooKeeper().delete(
|
||||
ZKUtil.joinZNode(watcher.getZNodePaths().rsZNode, rsToSuspend.getServerName().toString()),
|
||||
|
@ -137,7 +137,7 @@ public class TestCompactionInDeadRegionServer {
|
|||
for (RegionServerThread thread : UTIL.getHBaseCluster().getRegionServerThreads()) {
|
||||
HRegionServer rs = thread.getRegionServer();
|
||||
if (rs != rsToSuspend) {
|
||||
return !rs.getOnlineRegions(TABLE_NAME).isEmpty();
|
||||
return !rs.getRegions(TABLE_NAME).isEmpty();
|
||||
}
|
||||
}
|
||||
return false;
|
||||
|
|
|
@ -138,7 +138,7 @@ public class TestCompactionState {
|
|||
ht = TEST_UTIL.createTable(table, families);
|
||||
loadData(ht, families, 3000, flushes);
|
||||
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
|
||||
List<Region> regions = rs.getOnlineRegions(table);
|
||||
List<Region> regions = rs.getRegions(table);
|
||||
int countBefore = countStoreFilesInFamilies(regions, families);
|
||||
int countBeforeSingleFamily = countStoreFilesInFamily(regions, family);
|
||||
assertTrue(countBefore > 0); // there should be some data files
|
||||
|
|
|
@ -209,7 +209,7 @@ public class TestEncryptionKeyRotation {
|
|||
throws IOException, InterruptedException {
|
||||
boolean compacted = false;
|
||||
for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName)
|
||||
.getOnlineRegions(tableName)) {
|
||||
.getRegions(tableName)) {
|
||||
for (HStore store : ((HRegion) region).getStores()) {
|
||||
compacted = false;
|
||||
while (!compacted) {
|
||||
|
@ -235,7 +235,7 @@ public class TestEncryptionKeyRotation {
|
|||
private static List<Path> findStorefilePaths(TableName tableName) throws Exception {
|
||||
List<Path> paths = new ArrayList<>();
|
||||
for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName)
|
||||
.getOnlineRegions(tableName)) {
|
||||
.getRegions(tableName)) {
|
||||
for (HStore store : ((HRegion) region).getStores()) {
|
||||
for (HStoreFile storefile : store.getStorefiles()) {
|
||||
paths.add(storefile.getPath());
|
||||
|
@ -248,7 +248,7 @@ public class TestEncryptionKeyRotation {
|
|||
private static List<Path> findCompactedStorefilePaths(TableName tableName) throws Exception {
|
||||
List<Path> paths = new ArrayList<>();
|
||||
for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName)
|
||||
.getOnlineRegions(tableName)) {
|
||||
.getRegions(tableName)) {
|
||||
for (HStore store : ((HRegion) region).getStores()) {
|
||||
Collection<HStoreFile> compactedfiles =
|
||||
store.getStoreEngine().getStoreFileManager().getCompactedfiles();
|
||||
|
|
|
@ -54,7 +54,7 @@ public class TestEncryptionRandomKeying {
|
|||
private static List<Path> findStorefilePaths(TableName tableName) throws Exception {
|
||||
List<Path> paths = new ArrayList<>();
|
||||
for (Region region:
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) {
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegions(htd.getTableName())) {
|
||||
for (HStore store : ((HRegion) region).getStores()) {
|
||||
for (HStoreFile storefile : store.getStorefiles()) {
|
||||
paths.add(storefile.getPath());
|
||||
|
|
|
@ -181,7 +181,7 @@ public class TestHRegionReplayEvents {
|
|||
primaryRegion.close();
|
||||
List<Region> regions = new ArrayList<>();
|
||||
regions.add(primaryRegion);
|
||||
when(rss.getOnlineRegions()).thenReturn(regions);
|
||||
when(rss.getRegions()).thenReturn(regions);
|
||||
|
||||
primaryRegion = HRegion.openHRegion(rootDir, primaryHri, htd, walPrimary, CONF, rss, null);
|
||||
secondaryRegion = HRegion.openHRegion(secondaryHri, htd, null, CONF, rss, null);
|
||||
|
@ -1394,7 +1394,7 @@ public class TestHRegionReplayEvents {
|
|||
primaryRegion.compactStores();
|
||||
List<Region> regions = new ArrayList<>();
|
||||
regions.add(primaryRegion);
|
||||
when(rss.getOnlineRegions()).thenReturn(regions);
|
||||
when(rss.getRegions()).thenReturn(regions);
|
||||
CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, rss, false);
|
||||
cleaner.chore();
|
||||
secondaryRegion.refreshStoreFiles();
|
||||
|
|
|
@ -330,7 +330,7 @@ public class TestPerColumnFamilyFlush {
|
|||
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
|
||||
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
||||
HRegionServer hrs = rsts.get(i).getRegionServer();
|
||||
for (Region region : hrs.getOnlineRegions(tableName)) {
|
||||
for (Region region : hrs.getRegions(tableName)) {
|
||||
return Pair.newPair(region, hrs);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -114,7 +114,7 @@ public class TestRegionFavoredNodes {
|
|||
// them as favored nodes through the region.
|
||||
for (int i = 0; i < REGION_SERVERS; i++) {
|
||||
HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(i);
|
||||
List<Region> regions = server.getOnlineRegions(TABLE_NAME);
|
||||
List<Region> regions = server.getRegions(TABLE_NAME);
|
||||
for (Region region : regions) {
|
||||
List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName>favoredNodes =
|
||||
new ArrayList<>(3);
|
||||
|
@ -142,7 +142,7 @@ public class TestRegionFavoredNodes {
|
|||
// they are consistent with the favored nodes for that region.
|
||||
for (int i = 0; i < REGION_SERVERS; i++) {
|
||||
HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(i);
|
||||
List<Region> regions = server.getOnlineRegions(TABLE_NAME);
|
||||
List<Region> regions = server.getRegions(TABLE_NAME);
|
||||
for (Region region : regions) {
|
||||
List<String> files = region.getStoreFileList(new byte[][]{COLUMN_FAMILY});
|
||||
for (String file : files) {
|
||||
|
|
|
@ -185,7 +185,7 @@ public class TestRegionReplicaFailover {
|
|||
// read from it the same data from primary and secondaries
|
||||
boolean aborted = false;
|
||||
for (RegionServerThread rs : HTU.getMiniHBaseCluster().getRegionServerThreads()) {
|
||||
for (Region r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) {
|
||||
for (Region r : rs.getRegionServer().getRegions(htd.getTableName())) {
|
||||
if (r.getRegionInfo().getReplicaId() == 0) {
|
||||
LOG.info("Aborting region server hosting primary region replica");
|
||||
rs.getRegionServer().abort("for test");
|
||||
|
@ -247,7 +247,7 @@ public class TestRegionReplicaFailover {
|
|||
// read from it the same data
|
||||
boolean aborted = false;
|
||||
for (RegionServerThread rs : HTU.getMiniHBaseCluster().getRegionServerThreads()) {
|
||||
for (Region r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) {
|
||||
for (Region r : rs.getRegionServer().getRegions(htd.getTableName())) {
|
||||
if (r.getRegionInfo().getReplicaId() == 1) {
|
||||
LOG.info("Aborting region server hosting secondary region replica");
|
||||
rs.getRegionServer().abort("for test");
|
||||
|
@ -308,7 +308,7 @@ public class TestRegionReplicaFailover {
|
|||
try {
|
||||
boolean aborted = false;
|
||||
for (RegionServerThread rs : HTU.getMiniHBaseCluster().getRegionServerThreads()) {
|
||||
for (Region r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) {
|
||||
for (Region r : rs.getRegionServer().getRegions(htd.getTableName())) {
|
||||
if (r.getRegionInfo().getReplicaId() == 1) {
|
||||
LOG.info("Aborting region server hosting secondary region replica");
|
||||
rs.getRegionServer().abort("for test");
|
||||
|
|
|
@ -162,7 +162,7 @@ public class TestRegionReplicas {
|
|||
openRegion(HTU, getRS(), hriSecondary);
|
||||
|
||||
// first try directly against region
|
||||
region = getRS().getFromOnlineRegions(hriSecondary.getEncodedName());
|
||||
region = getRS().getRegion(hriSecondary.getEncodedName());
|
||||
assertGet(region, 42, true);
|
||||
|
||||
assertGetRpc(hriSecondary, 42, true);
|
||||
|
@ -259,7 +259,7 @@ public class TestRegionReplicas {
|
|||
Threads.sleep(4 * refreshPeriod);
|
||||
|
||||
LOG.info("Checking results from secondary region replica");
|
||||
Region secondaryRegion = getRS().getFromOnlineRegions(hriSecondary.getEncodedName());
|
||||
Region secondaryRegion = getRS().getRegion(hriSecondary.getEncodedName());
|
||||
Assert.assertEquals(1, secondaryRegion.getStore(f).getStorefilesCount());
|
||||
|
||||
assertGet(secondaryRegion, 42, true);
|
||||
|
@ -446,11 +446,11 @@ public class TestRegionReplicas {
|
|||
region.flush(true);
|
||||
}
|
||||
|
||||
Region primaryRegion = getRS().getFromOnlineRegions(hriPrimary.getEncodedName());
|
||||
Region primaryRegion = getRS().getRegion(hriPrimary.getEncodedName());
|
||||
Assert.assertEquals(3, primaryRegion.getStore(f).getStorefilesCount());
|
||||
|
||||
// Refresh store files on the secondary
|
||||
Region secondaryRegion = getRS().getFromOnlineRegions(hriSecondary.getEncodedName());
|
||||
Region secondaryRegion = getRS().getRegion(hriSecondary.getEncodedName());
|
||||
secondaryRegion.getStore(f).refreshStoreFiles();
|
||||
Assert.assertEquals(3, secondaryRegion.getStore(f).getStorefilesCount());
|
||||
|
||||
|
|
|
@ -168,7 +168,7 @@ public class TestRegionServerAbort {
|
|||
public void prePut(ObserverContext<RegionCoprocessorEnvironment> c, Put put, WALEdit edit,
|
||||
Durability durability) throws IOException {
|
||||
if (put.getAttribute(DO_ABORT) != null) {
|
||||
HRegionServer rs = (HRegionServer) c.getEnvironment().getRegionServerServices();
|
||||
HRegionServer rs = (HRegionServer) c.getEnvironment().getCoprocessorRegionServerServices();
|
||||
LOG.info("Triggering abort for regionserver " + rs.getServerName());
|
||||
rs.abort("Aborting for test");
|
||||
}
|
||||
|
|
|
@ -496,7 +496,7 @@ public class TestRegionServerMetrics {
|
|||
byte[] val = Bytes.toBytes("mobdata");
|
||||
try {
|
||||
Table table = TEST_UTIL.createTable(htd, new byte[0][0], conf);
|
||||
Region region = rs.getOnlineRegions(tableName).get(0);
|
||||
Region region = rs.getRegions(tableName).get(0);
|
||||
for (int insertCount = 0; insertCount < numHfiles; insertCount++) {
|
||||
Put p = new Put(Bytes.toBytes(insertCount));
|
||||
p.addColumn(cf, qualifier, val);
|
||||
|
|
|
@ -105,7 +105,7 @@ public class TestRegionSplitPolicy {
|
|||
// return 'online regions'.
|
||||
RegionServerServices rss = Mockito.mock(RegionServerServices.class);
|
||||
final List<Region> regions = new ArrayList<>();
|
||||
Mockito.when(rss.getOnlineRegions(TABLENAME)).thenReturn(regions);
|
||||
Mockito.when(rss.getRegions(TABLENAME)).thenReturn(regions);
|
||||
Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss);
|
||||
// Set max size for this 'table'.
|
||||
long maxSplitSize = 1024L;
|
||||
|
@ -164,7 +164,7 @@ public class TestRegionSplitPolicy {
|
|||
|
||||
RegionServerServices rss = Mockito.mock(RegionServerServices.class);
|
||||
final List<Region> regions = new ArrayList<>();
|
||||
Mockito.when(rss.getOnlineRegions(TABLENAME)).thenReturn(regions);
|
||||
Mockito.when(rss.getRegions(TABLENAME)).thenReturn(regions);
|
||||
Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss);
|
||||
Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(0L);
|
||||
Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(0L);
|
||||
|
|
|
@ -95,7 +95,7 @@ public class TestSplitWalDataLoss {
|
|||
@Test
|
||||
public void test() throws IOException, InterruptedException {
|
||||
final HRegionServer rs = testUtil.getRSForFirstRegionInTable(tableName);
|
||||
final HRegion region = (HRegion) rs.getOnlineRegions(tableName).get(0);
|
||||
final HRegion region = (HRegion) rs.getRegions(tableName).get(0);
|
||||
HRegion spiedRegion = spy(region);
|
||||
final MutableBoolean flushed = new MutableBoolean(false);
|
||||
final MutableBoolean reported = new MutableBoolean(false);
|
||||
|
|
|
@ -78,7 +78,7 @@ public class TestCompactedHFilesDischarger {
|
|||
rss = mock(RegionServerServices.class);
|
||||
List<Region> regions = new ArrayList<>(1);
|
||||
regions.add(region);
|
||||
when(rss.getOnlineRegions()).thenReturn(regions);
|
||||
when(rss.getRegions()).thenReturn(regions);
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
|
@ -70,7 +70,7 @@ public class TestFIFOCompactionPolicy {
|
|||
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
|
||||
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
||||
HRegionServer hrs = rsts.get(i).getRegionServer();
|
||||
for (Region region : hrs.getOnlineRegions(tableName)) {
|
||||
for (Region region : hrs.getRegions(tableName)) {
|
||||
return region.getStores().iterator().next();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -76,7 +76,7 @@ public class TestCompactionWithThroughputController {
|
|||
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
|
||||
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
||||
HRegionServer hrs = rsts.get(i).getRegionServer();
|
||||
for (Region region : hrs.getOnlineRegions(tableName)) {
|
||||
for (Region region : hrs.getRegions(tableName)) {
|
||||
return region.getStores().iterator().next();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -78,7 +78,7 @@ public class TestFlushWithThroughputController {
|
|||
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
|
||||
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
||||
HRegionServer hrs = rsts.get(i).getRegionServer();
|
||||
for (Region region : hrs.getOnlineRegions(tableName)) {
|
||||
for (Region region : hrs.getRegions(tableName)) {
|
||||
return region.getStores().iterator().next();
|
||||
}
|
||||
}
|
||||
|
@ -165,7 +165,7 @@ public class TestFlushWithThroughputController {
|
|||
HRegionServer regionServer = hbtu.getRSForFirstRegionInTable(tableName);
|
||||
PressureAwareFlushThroughputController throughputController =
|
||||
(PressureAwareFlushThroughputController) regionServer.getFlushThroughputController();
|
||||
for (Region region : regionServer.getOnlineRegions()) {
|
||||
for (Region region : regionServer.getRegions()) {
|
||||
region.flush(true);
|
||||
}
|
||||
assertEquals(0.0, regionServer.getFlushPressure(), EPSILON);
|
||||
|
|
|
@ -189,7 +189,7 @@ public abstract class AbstractTestLogRolling {
|
|||
this.tableName = getName();
|
||||
// TODO: Why does this write data take for ever?
|
||||
startAndWriteData();
|
||||
HRegionInfo region = server.getOnlineRegions(TableName.valueOf(tableName)).get(0)
|
||||
HRegionInfo region = server.getRegions(TableName.valueOf(tableName)).get(0)
|
||||
.getRegionInfo();
|
||||
final WAL log = server.getWAL(region);
|
||||
LOG.info("after writing there are " + AbstractFSWALProvider.getNumRolledLogFiles(log) + " log files");
|
||||
|
@ -249,7 +249,7 @@ public abstract class AbstractTestLogRolling {
|
|||
table = createTestTable(getName());
|
||||
|
||||
server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
|
||||
Region region = server.getOnlineRegions(table.getName()).get(0);
|
||||
Region region = server.getRegions(table.getName()).get(0);
|
||||
final WAL log = server.getWAL(region.getRegionInfo());
|
||||
Store s = region.getStore(HConstants.CATALOG_FAMILY);
|
||||
|
||||
|
|
|
@ -58,7 +58,7 @@ public class TestAsyncLogRolling extends AbstractTestLogRolling {
|
|||
TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
|
||||
doPut(table, 1);
|
||||
server = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
|
||||
HRegionInfo hri = server.getOnlineRegions(table.getName()).get(0).getRegionInfo();
|
||||
HRegionInfo hri = server.getRegions(table.getName()).get(0).getRegionInfo();
|
||||
AsyncFSWAL wal = (AsyncFSWAL) server.getWAL(hri);
|
||||
int numRolledLogFiles = AsyncFSWALProvider.getNumRolledLogFiles(wal);
|
||||
DatanodeInfo[] dnInfos = wal.getPipeline();
|
||||
|
|
|
@ -140,7 +140,7 @@ public class TestLogRolling extends AbstractTestLogRolling {
|
|||
Table table = TEST_UTIL.getConnection().getTable(desc.getTableName());
|
||||
|
||||
server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName());
|
||||
HRegionInfo region = server.getOnlineRegions(desc.getTableName()).get(0).getRegionInfo();
|
||||
HRegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo();
|
||||
final FSHLog log = (FSHLog) server.getWAL(region);
|
||||
final AtomicBoolean lowReplicationHookCalled = new AtomicBoolean(false);
|
||||
|
||||
|
@ -248,7 +248,7 @@ public class TestLogRolling extends AbstractTestLogRolling {
|
|||
Table table = TEST_UTIL.getConnection().getTable(desc.getTableName());
|
||||
|
||||
server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName());
|
||||
HRegionInfo region = server.getOnlineRegions(desc.getTableName()).get(0).getRegionInfo();
|
||||
HRegionInfo region = server.getRegions(desc.getTableName()).get(0).getRegionInfo();
|
||||
final WAL log = server.getWAL(region);
|
||||
final List<Path> paths = new ArrayList<>(1);
|
||||
final List<Integer> preLogRolledCalled = new ArrayList<>();
|
||||
|
|
|
@ -247,7 +247,7 @@ public class TestRegionReplicaReplicationEndpoint {
|
|||
|
||||
for (int i=0; i < NB_SERVERS; i++) {
|
||||
HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(i);
|
||||
List<Region> onlineRegions = rs.getOnlineRegions(tableName);
|
||||
List<Region> onlineRegions = rs.getRegions(tableName);
|
||||
for (Region region : onlineRegions) {
|
||||
regions[region.getRegionInfo().getReplicaId()] = region;
|
||||
}
|
||||
|
|
|
@ -176,7 +176,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
|
|||
// replay the edits to the secondary using replay callable
|
||||
replicateUsingCallable(connection, entries);
|
||||
|
||||
Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName());
|
||||
Region region = rs0.getRegion(hriSecondary.getEncodedName());
|
||||
HTU.verifyNumericRows(region, f, 0, 1000);
|
||||
|
||||
HTU.deleteNumericRows(table, f, 0, 1000);
|
||||
|
@ -216,7 +216,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
|
|||
// replay the edits to the secondary using replay callable
|
||||
replicateUsingCallable(connection, entries);
|
||||
|
||||
Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName());
|
||||
Region region = rs0.getRegion(hriSecondary.getEncodedName());
|
||||
HTU.verifyNumericRows(region, f, 0, 1000);
|
||||
|
||||
HTU.loadNumericRows(table, f, 1000, 2000); // load some more data to primary
|
||||
|
@ -228,7 +228,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
|
|||
// replicate the new data
|
||||
replicateUsingCallable(connection, entries);
|
||||
|
||||
region = rs1.getFromOnlineRegions(hriSecondary.getEncodedName());
|
||||
region = rs1.getRegion(hriSecondary.getEncodedName());
|
||||
// verify the new data. old data may or may not be there
|
||||
HTU.verifyNumericRows(region, f, 1000, 2000);
|
||||
|
||||
|
@ -261,7 +261,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
|
|||
replicator.replicate(new ReplicateContext().setEntries(Lists.newArrayList(entries))
|
||||
.setWalGroupId(fakeWalGroupId));
|
||||
|
||||
Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName());
|
||||
Region region = rs0.getRegion(hriSecondary.getEncodedName());
|
||||
HTU.verifyNumericRows(region, f, 0, 1000);
|
||||
|
||||
HTU.deleteNumericRows(table, f, 0, 1000);
|
||||
|
|
|
@ -2148,7 +2148,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
|
||||
final int RETRIES_LIMIT = 10;
|
||||
int retries = 0;
|
||||
while (newRs.getOnlineRegions(TEST_TABLE2).size() < 1 && retries < RETRIES_LIMIT) {
|
||||
while (newRs.getRegions(TEST_TABLE2).size() < 1 && retries < RETRIES_LIMIT) {
|
||||
LOG.debug("Waiting for region to be opened. Already retried " + retries
|
||||
+ " times.");
|
||||
try {
|
||||
|
@ -2609,7 +2609,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
for (JVMClusterUtil.RegionServerThread thread:
|
||||
TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
|
||||
HRegionServer rs = thread.getRegionServer();
|
||||
for (Region region: rs.getOnlineRegions(TEST_TABLE)) {
|
||||
for (Region region: rs.getRegions(TEST_TABLE)) {
|
||||
region.getCoprocessorHost().load(PingCoprocessor.class,
|
||||
Coprocessor.PRIORITY_USER, conf);
|
||||
}
|
||||
|
|
|
@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
|||
import org.apache.hadoop.hbase.ipc.SimpleRpcServer;
|
||||
import org.apache.hadoop.hbase.metrics.MetricRegistry;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.CoprocessorRegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.security.SecurityInfo;
|
||||
|
@ -265,7 +266,7 @@ public class TestTokenAuthentication {
|
|||
public HRegion getRegion() { return null; }
|
||||
|
||||
@Override
|
||||
public RegionServerServices getRegionServerServices() {
|
||||
public CoprocessorRegionServerServices getCoprocessorRegionServerServices() {
|
||||
return mockServices;
|
||||
}
|
||||
|
||||
|
|
|
@ -307,7 +307,7 @@ public abstract class TestVisibilityLabels {
|
|||
List<RegionServerThread> regionServerThreads = TEST_UTIL.getHBaseCluster()
|
||||
.getRegionServerThreads();
|
||||
for (RegionServerThread rsThread : regionServerThreads) {
|
||||
List<Region> onlineRegions = rsThread.getRegionServer().getOnlineRegions(
|
||||
List<Region> onlineRegions = rsThread.getRegionServer().getRegions(
|
||||
LABELS_TABLE_NAME);
|
||||
if (onlineRegions.size() > 0) {
|
||||
rsThread.getRegionServer().abort("Aborting ");
|
||||
|
@ -341,7 +341,7 @@ public abstract class TestVisibilityLabels {
|
|||
for (RegionServerThread rsThread : regionServerThreads) {
|
||||
while (true) {
|
||||
if (!rsThread.getRegionServer().isAborted()) {
|
||||
List<Region> onlineRegions = rsThread.getRegionServer().getOnlineRegions(
|
||||
List<Region> onlineRegions = rsThread.getRegionServer().getRegions(
|
||||
LABELS_TABLE_NAME);
|
||||
if (onlineRegions.size() > 0) {
|
||||
break;
|
||||
|
@ -392,13 +392,13 @@ public abstract class TestVisibilityLabels {
|
|||
} catch (InterruptedException e) {
|
||||
}
|
||||
}
|
||||
while (regionServer.getOnlineRegions(LABELS_TABLE_NAME).isEmpty()) {
|
||||
while (regionServer.getRegions(LABELS_TABLE_NAME).isEmpty()) {
|
||||
try {
|
||||
Thread.sleep(10);
|
||||
} catch (InterruptedException e) {
|
||||
}
|
||||
}
|
||||
Region labelsTableRegion = regionServer.getOnlineRegions(LABELS_TABLE_NAME).get(0);
|
||||
Region labelsTableRegion = regionServer.getRegions(LABELS_TABLE_NAME).get(0);
|
||||
while (labelsTableRegion.isRecovering()) {
|
||||
try {
|
||||
Thread.sleep(10);
|
||||
|
|
|
@ -754,7 +754,7 @@ public final class SnapshotTestingUtils {
|
|||
final TableName tableName)
|
||||
throws IOException, InterruptedException {
|
||||
HRegionServer rs = util.getRSForFirstRegionInTable(tableName);
|
||||
List<Region> onlineRegions = rs.getOnlineRegions(tableName);
|
||||
List<Region> onlineRegions = rs.getRegions(tableName);
|
||||
for (Region region : onlineRegions) {
|
||||
region.waitForFlushesAndCompactions();
|
||||
}
|
||||
|
|
|
@ -142,7 +142,7 @@ public class TestHBaseFsckEncryption {
|
|||
private List<Path> findStorefilePaths(TableName tableName) throws Exception {
|
||||
List<Path> paths = new ArrayList<>();
|
||||
for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName)
|
||||
.getOnlineRegions(htd.getTableName())) {
|
||||
.getRegions(htd.getTableName())) {
|
||||
for (HStore store : ((HRegion) region).getStores()) {
|
||||
for (HStoreFile storefile : store.getStorefiles()) {
|
||||
paths.add(storefile.getPath());
|
||||
|
|
|
@ -451,7 +451,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
|
|||
// flakiness of this test.
|
||||
HRegion r = HRegion.openHRegion(
|
||||
region, htdDisabled, hrs.getWAL(region), conf);
|
||||
hrs.addToOnlineRegions(r);
|
||||
hrs.addRegion(r);
|
||||
|
||||
HBaseFsck hbck = doFsck(conf, false);
|
||||
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
|
||||
|
|
|
@ -130,7 +130,7 @@ public class TestWALFiltering {
|
|||
private List<byte[]> getRegionsByServer(int rsId) throws IOException {
|
||||
List<byte[]> regionNames = Lists.newArrayList();
|
||||
HRegionServer hrs = getRegionServer(rsId);
|
||||
for (Region r : hrs.getOnlineRegions(TABLE_NAME)) {
|
||||
for (Region r : hrs.getRegions(TABLE_NAME)) {
|
||||
regionNames.add(r.getRegionInfo().getRegionName());
|
||||
}
|
||||
return regionNames;
|
||||
|
|
|
@ -62,7 +62,7 @@ public class ErrorThrowingGetObserver implements RegionObserver {
|
|||
throw new NotServingRegionException("Failing for test");
|
||||
case REGION_MOVED:
|
||||
throw new RegionMovedException(
|
||||
e.getEnvironment().getRegionServerServices().getServerName(), 1);
|
||||
e.getEnvironment().getCoprocessorRegionServerServices().getServerName(), 1);
|
||||
case SCANNER_RESET:
|
||||
throw new ScannerResetException("Failing for test");
|
||||
case UNKNOWN_SCANNER:
|
||||
|
|
Loading…
Reference in New Issue