HBASE-10569 Co-locate meta and master

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1581479 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
jxiang 2014-03-25 19:34:52 +00:00
parent 0e5d21e96a
commit 7716a3ca94
110 changed files with 5088 additions and 4918 deletions

View File

@ -129,7 +129,7 @@ public class CatalogTracker {
* @throws IOException
*/
public CatalogTracker(final Configuration conf) throws IOException {
this(null, conf, null);
this(null, conf, HConnectionManager.getConnection(conf), null);
}
/**
@ -144,12 +144,6 @@ public class CatalogTracker {
* {@link Configuration} as our Abortable.
* @throws IOException
*/
public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
Abortable abortable)
throws IOException {
this(zk, conf, HConnectionManager.getConnection(conf), abortable);
}
public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf,
HConnection connection, Abortable abortable)
throws IOException {
@ -157,6 +151,8 @@ public class CatalogTracker {
if (abortable == null) {
// A connection is abortable.
this.abortable = this.connection;
} else {
this.abortable = abortable;
}
Abortable throwableAborter = new Abortable() {
@ -322,6 +318,7 @@ public class CatalogTracker {
* invocation, or may be null.
* @throws IOException
*/
@SuppressWarnings("deprecation")
private AdminService.BlockingInterface getCachedConnection(ServerName sn)
throws IOException {
if (sn == null) {

View File

@ -0,0 +1,409 @@
/**
* 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.client;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.ExecutorService;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
/**
* An internal class that adapts a {@link HConnection}.
* HConnection is created from HConnectionManager. The default
* implementation talks to region servers over RPC since it
* doesn't know if the connection is used by one region server
* itself. This adapter makes it possible to change some of the
* default logic. Especially, when the connection is used
* internally by some the region server.
*
* @see ConnectionUtils#createShortCircuitHConnection(HConnection, ServerName,
* AdminService.BlockingInterface, ClientService.BlockingInterface)
*/
@InterfaceAudience.Private
@SuppressWarnings("deprecation")
//NOTE: DO NOT make this class public. It was made package-private on purpose.
class ConnectionAdapter implements ClusterConnection {
private final ClusterConnection wrappedConnection;
public ConnectionAdapter(HConnection c) {
wrappedConnection = (ClusterConnection)c;
}
@Override
public void abort(String why, Throwable e) {
wrappedConnection.abort(why, e);
}
@Override
public boolean isAborted() {
return wrappedConnection.isAborted();
}
@Override
public void close() throws IOException {
wrappedConnection.close();
}
@Override
public Configuration getConfiguration() {
return wrappedConnection.getConfiguration();
}
@Override
public HTableInterface getTable(String tableName) throws IOException {
return wrappedConnection.getTable(tableName);
}
@Override
public HTableInterface getTable(byte[] tableName) throws IOException {
return wrappedConnection.getTable(tableName);
}
@Override
public HTableInterface getTable(TableName tableName) throws IOException {
return wrappedConnection.getTable(tableName);
}
@Override
public HTableInterface getTable(String tableName, ExecutorService pool)
throws IOException {
return wrappedConnection.getTable(tableName, pool);
}
@Override
public HTableInterface getTable(byte[] tableName, ExecutorService pool)
throws IOException {
return wrappedConnection.getTable(tableName, pool);
}
@Override
public HTableInterface getTable(TableName tableName, ExecutorService pool)
throws IOException {
return wrappedConnection.getTable(tableName, pool);
}
@Override
public boolean isMasterRunning() throws MasterNotRunningException,
ZooKeeperConnectionException {
return wrappedConnection.isMasterRunning();
}
@Override
public boolean isTableEnabled(TableName tableName) throws IOException {
return wrappedConnection.isTableEnabled(tableName);
}
@Override
public boolean isTableEnabled(byte[] tableName) throws IOException {
return wrappedConnection.isTableEnabled(tableName);
}
@Override
public boolean isTableDisabled(TableName tableName) throws IOException {
return wrappedConnection.isTableDisabled(tableName);
}
@Override
public boolean isTableDisabled(byte[] tableName) throws IOException {
return wrappedConnection.isTableDisabled(tableName);
}
@Override
public boolean isTableAvailable(TableName tableName) throws IOException {
return wrappedConnection.isTableAvailable(tableName);
}
@Override
public boolean isTableAvailable(byte[] tableName) throws IOException {
return wrappedConnection.isTableAvailable(tableName);
}
@Override
public boolean isTableAvailable(TableName tableName, byte[][] splitKeys)
throws IOException {
return wrappedConnection.isTableAvailable(tableName, splitKeys);
}
@Override
public boolean isTableAvailable(byte[] tableName, byte[][] splitKeys)
throws IOException {
return wrappedConnection.isTableAvailable(tableName, splitKeys);
}
@Override
public HTableDescriptor[] listTables() throws IOException {
return wrappedConnection.listTables();
}
@Override
public String[] getTableNames() throws IOException {
return wrappedConnection.getTableNames();
}
@Override
public TableName[] listTableNames() throws IOException {
return wrappedConnection.listTableNames();
}
@Override
public HTableDescriptor getHTableDescriptor(TableName tableName)
throws IOException {
return wrappedConnection.getHTableDescriptor(tableName);
}
@Override
public HTableDescriptor getHTableDescriptor(byte[] tableName)
throws IOException {
return wrappedConnection.getHTableDescriptor(tableName);
}
@Override
public HRegionLocation locateRegion(TableName tableName, byte[] row)
throws IOException {
return wrappedConnection.locateRegion(tableName, row);
}
@Override
public HRegionLocation locateRegion(byte[] tableName, byte[] row)
throws IOException {
return wrappedConnection.locateRegion(tableName, row);
}
@Override
public void clearRegionCache() {
wrappedConnection.clearRegionCache();
}
@Override
public void clearRegionCache(TableName tableName) {
wrappedConnection.clearRegionCache(tableName);
}
@Override
public void clearRegionCache(byte[] tableName) {
wrappedConnection.clearRegionCache(tableName);
}
@Override
public void deleteCachedRegionLocation(HRegionLocation location) {
wrappedConnection.deleteCachedRegionLocation(location);
}
@Override
public HRegionLocation relocateRegion(TableName tableName, byte[] row)
throws IOException {
return wrappedConnection.relocateRegion(tableName, row);
}
@Override
public HRegionLocation relocateRegion(byte[] tableName, byte[] row)
throws IOException {
return wrappedConnection.relocateRegion(tableName, row);
}
@Override
public void updateCachedLocations(TableName tableName, byte[] rowkey,
Object exception, HRegionLocation source) {
wrappedConnection.updateCachedLocations(tableName, rowkey, exception, source);
}
@Override
public void updateCachedLocations(TableName tableName, byte[] rowkey,
Object exception, ServerName source) {
wrappedConnection.updateCachedLocations(tableName, rowkey, exception, source);
}
@Override
public void updateCachedLocations(byte[] tableName, byte[] rowkey,
Object exception, HRegionLocation source) {
wrappedConnection.updateCachedLocations(tableName, rowkey, exception, source);
}
@Override
public HRegionLocation locateRegion(byte[] regionName) throws IOException {
return wrappedConnection.locateRegion(regionName);
}
@Override
public List<HRegionLocation> locateRegions(TableName tableName)
throws IOException {
return wrappedConnection.locateRegions(tableName);
}
@Override
public List<HRegionLocation> locateRegions(byte[] tableName)
throws IOException {
return wrappedConnection.locateRegions(tableName);
}
@Override
public List<HRegionLocation> locateRegions(TableName tableName,
boolean useCache, boolean offlined) throws IOException {
return wrappedConnection.locateRegions(tableName, useCache, offlined);
}
@Override
public List<HRegionLocation> locateRegions(byte[] tableName,
boolean useCache, boolean offlined) throws IOException {
return wrappedConnection.locateRegions(tableName, useCache, offlined);
}
@Override
public MasterService.BlockingInterface getMaster() throws IOException {
return wrappedConnection.getMaster();
}
@Override
public AdminService.BlockingInterface getAdmin(
ServerName serverName) throws IOException {
return wrappedConnection.getAdmin(serverName);
}
@Override
public ClientService.BlockingInterface getClient(
ServerName serverName) throws IOException {
return wrappedConnection.getClient(serverName);
}
@Override
public AdminService.BlockingInterface getAdmin(
ServerName serverName, boolean getMaster) throws IOException {
return wrappedConnection.getAdmin(serverName, getMaster);
}
@Override
public HRegionLocation getRegionLocation(TableName tableName, byte[] row,
boolean reload) throws IOException {
return wrappedConnection.getRegionLocation(tableName, row, reload);
}
@Override
public HRegionLocation getRegionLocation(byte[] tableName, byte[] row,
boolean reload) throws IOException {
return wrappedConnection.getRegionLocation(tableName, row, reload);
}
@Override
public void processBatch(List<? extends Row> actions, TableName tableName,
ExecutorService pool, Object[] results) throws IOException,
InterruptedException {
wrappedConnection.processBatch(actions, tableName, pool, results);
}
@Override
public void processBatch(List<? extends Row> actions, byte[] tableName,
ExecutorService pool, Object[] results) throws IOException,
InterruptedException {
wrappedConnection.processBatch(actions, tableName, pool, results);
}
@Override
public <R> void processBatchCallback(List<? extends Row> list,
TableName tableName, ExecutorService pool, Object[] results,
Callback<R> callback) throws IOException, InterruptedException {
wrappedConnection.processBatchCallback(list, tableName, pool, results, callback);
}
@Override
public <R> void processBatchCallback(List<? extends Row> list,
byte[] tableName, ExecutorService pool, Object[] results,
Callback<R> callback) throws IOException, InterruptedException {
wrappedConnection.processBatchCallback(list, tableName, pool, results, callback);
}
@Override
public void setRegionCachePrefetch(TableName tableName, boolean enable) {
wrappedConnection.setRegionCachePrefetch(tableName, enable);
}
@Override
public void setRegionCachePrefetch(byte[] tableName, boolean enable) {
wrappedConnection.setRegionCachePrefetch(tableName, enable);
}
@Override
public boolean getRegionCachePrefetch(TableName tableName) {
return wrappedConnection.getRegionCachePrefetch(tableName);
}
@Override
public boolean getRegionCachePrefetch(byte[] tableName) {
return wrappedConnection.getRegionCachePrefetch(tableName);
}
@Override
public int getCurrentNrHRS() throws IOException {
return wrappedConnection.getCurrentNrHRS();
}
@Override
public HTableDescriptor[] getHTableDescriptorsByTableName(
List<TableName> tableNames) throws IOException {
return wrappedConnection.getHTableDescriptorsByTableName(tableNames);
}
@Override
public HTableDescriptor[] getHTableDescriptors(List<String> tableNames)
throws IOException {
return wrappedConnection.getHTableDescriptors(tableNames);
}
@Override
public boolean isClosed() {
return wrappedConnection.isClosed();
}
@Override
public void clearCaches(ServerName sn) {
wrappedConnection.clearCaches(sn);
}
@Override
public MasterKeepAliveConnection getKeepAliveMasterService()
throws MasterNotRunningException {
return wrappedConnection.getKeepAliveMasterService();
}
@Override
public boolean isDeadServer(ServerName serverName) {
return wrappedConnection.isDeadServer(serverName);
}
@Override
public NonceGenerator getNonceGenerator() {
return wrappedConnection.getNonceGenerator();
}
@Override
public AsyncProcess getAsyncProcess() {
return wrappedConnection.getAsyncProcess();
}
}

View File

@ -17,12 +17,16 @@
*/
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
/**
* Utility used by client connections.
@ -92,4 +96,31 @@ public class ConnectionUtils {
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
log.debug(sn + " HConnection server-to-server retries=" + retries);
}
/**
* Adapt a HConnection so that it can bypass the RPC layer (serialization,
* deserialization, networking, etc..) when it talks to a local server.
* @param conn the connection to adapt
* @param serverName the local server name
* @param admin the admin interface of the local server
* @param client the client interface of the local server
* @return an adapted/decorated HConnection
*/
public static HConnection createShortCircuitHConnection(final HConnection conn,
final ServerName serverName, final AdminService.BlockingInterface admin,
final ClientService.BlockingInterface client) {
return new ConnectionAdapter(conn) {
@Override
public AdminService.BlockingInterface getAdmin(
ServerName sn, boolean getMaster) throws IOException {
return serverName.equals(sn) ? admin : super.getAdmin(sn, getMaster);
}
@Override
public ClientService.BlockingInterface getClient(
ServerName sn) throws IOException {
return serverName.equals(sn) ? client : super.getClient(sn);
}
};
}
}

View File

@ -136,7 +136,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
}
@Override
public MasterService.BlockingInterface getMaster()
public MasterService.BlockingInterface getMasterAdminService()
throws IOException {
HConnection conn = HConnectionManager.getConnection(conf);
return conn.getMaster();
@ -170,7 +170,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
long start = System.currentTimeMillis();
while (System.currentTimeMillis() - start < timeout) {
try {
getMaster();
getMasterAdminService();
return true;
} catch (MasterNotRunningException m) {
LOG.warn("Master not started yet " + m);

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.LinkedList;
import java.util.List;
@ -58,11 +59,25 @@ public class Action {
public void perform() throws Exception { }
/** Returns current region servers */
/** Returns current region servers - active master */
protected ServerName[] getCurrentServers() throws IOException {
Collection<ServerName> regionServers = cluster.getClusterStatus().getServers();
if (regionServers == null || regionServers.size() <= 0) return new ServerName [] {};
return regionServers.toArray(new ServerName[regionServers.size()]);
ClusterStatus clusterStatus = cluster.getClusterStatus();
Collection<ServerName> regionServers = clusterStatus.getServers();
int count = regionServers == null ? 0 : regionServers.size();
if (count <= 0) {
return new ServerName [] {};
}
ServerName master = clusterStatus.getMaster();
if (master == null || !regionServers.contains(master)) {
return regionServers.toArray(new ServerName[count]);
}
if (count == 1) {
return new ServerName [] {};
}
ArrayList<ServerName> tmp = new ArrayList<ServerName>(count);
tmp.addAll(regionServers);
tmp.remove(master);
return tmp.toArray(new ServerName[count-1]);
}
protected void killMaster(ServerName server) throws IOException {

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.chaos.actions;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ServerName;
/**
@ -35,6 +36,12 @@ public class RestartRsHoldingMetaAction extends RestartActionBaseAction {
LOG.warn("No server is holding hbase:meta right now.");
return;
}
restartRs(server, sleepTime);
ClusterStatus clusterStatus = cluster.getClusterStatus();
if (server.equals(clusterStatus.getMaster())) {
// Master holds the meta, so restart the master.
restartMaster(server, sleepTime);
} else {
restartRs(server, sleepTime);
}
}
}

View File

@ -40,15 +40,14 @@ if (master.isActiveMaster()) {
<%java>
ServerName [] serverNames = masters.toArray(new ServerName[masters.size()]);
int infoPort = master.getConfiguration().getInt("hbase.master.info.port", 16010);
</%java>
<%if (!master.isActiveMaster()) %>
<%if serverNames[0] != null %>
<h2>Master</h2>
<a href="//<% serverNames[0].getHostname() %>:
<% master.getConfiguration().getInt("hbase.master.info.port", 16010) %>/master-status"
target="_blank">
<% serverNames[0].getHostname() %>
</a>
<a href="//<% serverNames[0].getHostname() %>:<%
infoPort %>/master-status" target="_blank"><%
serverNames[0].getHostname() %></a>
<%else>
Unable to parse master hostname.
</%if>
@ -66,11 +65,10 @@ ServerName [] serverNames = masters.toArray(new ServerName[masters.size()]);
for (ServerName serverName : serverNames) {
</%java>
<tr>
<td><a href="//<% serverName.getHostname() %>:
<% master.getConfiguration().getInt("hbase.master.info.port", 16010) %>/master-status"
target="_blank">
<% serverName.getHostname() %></a>
</td>
<td><a href="//<% serverName.getHostname() %>:<%
infoPort %>/master-status" target="_blank"><%
serverName.getHostname() %></a>
</td>
<td><% serverName.getPort() %></td>
<td><% new Date(serverName.getStartcode()) %></td>
</tr>

View File

@ -63,7 +63,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
<%class>
public String formatZKString() {
StringBuilder quorums = new StringBuilder();
String zkQuorum = master.getZooKeeperWatcher().getQuorum();
String zkQuorum = master.getZooKeeper().getQuorum();
if (null == zkQuorum) {
return quorums.toString();
@ -278,7 +278,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
</%if>
<tr>
<td>Coprocessors</td>
<td><% java.util.Arrays.toString(master.getCoprocessors()) %></td>
<td><% java.util.Arrays.toString(master.getMasterCoprocessors()) %></td>
<td>Coprocessors currently loaded by the master</td>
</tr>
</%if>

View File

@ -283,7 +283,7 @@ if (sl.getTotalCompactingKVs() > 0) {
</%args>
<%java>
int infoPort = master.getRegionServerInfoPort(serverName);
String url = "//" + serverName.getHostname() + ":" + infoPort + "/";
String url = "//" + serverName.getHostname() + ":" + infoPort + "/rs-status";
</%java>
<%if (infoPort > 0) %>

View File

@ -36,10 +36,10 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
<%java return; %>
</%if>
<%java>
ServerInfo serverInfo = ProtobufUtil.getServerInfo(regionServer);
ServerInfo serverInfo = ProtobufUtil.getServerInfo(regionServer.getRSRpcServices());
ServerName serverName = ProtobufUtil.toServerName(serverInfo.getServerName());
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(regionServer);
int masterInfoPort = regionServer.getConfiguration().getInt("hbase.master.info.port", 16010);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices());
int infoPort = regionServer.getConfiguration().getInt("hbase.master.info.port", 16010);
MasterAddressTracker masterAddressTracker = regionServer.getMasterAddressTracker();
ServerName masterServerName = masterAddressTracker == null ? null
: masterAddressTracker.getMasterAddress();
@ -98,7 +98,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
<section>
<h2>Server Metrics</h2>
<& ServerMetricsTmpl; mWrap = regionServer.getMetrics().getRegionServerWrapper(); &>
<& ServerMetricsTmpl; mWrap = regionServer.getRegionServerMetrics().getRegionServerWrapper(); &>
</section>
<section>
@ -135,7 +135,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
</tr>
<tr>
<td>Coprocessors</td>
<td><% java.util.Arrays.toString(regionServer.getCoprocessors()) %></td>
<td><% java.util.Arrays.toString(regionServer.getRegionServerCoprocessors()) %></td>
<td>Coprocessors currently loaded by this regionserver</td>
</tr>
<tr>
@ -146,14 +146,12 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
<tr>
<td>HBase Master</td>
<td>
<%if (masterInfoPort < 0) %>
No hbase.master.info.port found
<%elseif masterServerName == null %>
<%if masterServerName == null %>
No master found
<%else>
<%java>
String host = masterServerName.getHostname() + ":" + masterInfoPort;
String url = "//" + host + "/";
String host = masterServerName.getHostname() + ":" + infoPort;
String url = "//" + host + "/master-status";
</%java>
<a href="<% url %>"><% host %></a>
</%if>

View File

@ -32,25 +32,12 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.BlockingService;
import com.google.protobuf.Descriptors.MethodDescriptor;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
/**
* Connection to an HTable from within a Coprocessor. We can do some nice tricks since we know we
@ -106,28 +93,7 @@ public class CoprocessorHConnection implements ClusterConnection {
}
// the client is attempting to write to the same regionserver, we can short-circuit to our
// local regionserver
final BlockingService blocking = ClientService.newReflectiveBlockingService(this.server);
final RpcServerInterface rpc = this.server.getRpcServer();
final MonitoredRPCHandler status =
TaskMonitor.get().createRPCStatus(Thread.currentThread().getName());
status.pause("Setting up server-local call");
final long timestamp = EnvironmentEdgeManager.currentTimeMillis();
BlockingRpcChannel channel = new BlockingRpcChannel() {
@Override
public Message callBlockingMethod(MethodDescriptor method, RpcController controller,
Message request, Message responsePrototype) throws ServiceException {
try {
// we never need a cell-scanner - everything is already fully formed
return rpc.call(blocking, method, request, null, timestamp, status).getFirst();
} catch (IOException e) {
throw new ServiceException(e);
}
}
};
return ClientService.newBlockingStub(channel);
return server.getRSRpcServices();
}
public void abort(String why, Throwable e) {

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.hbase.ipc;
import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
@ -121,7 +121,6 @@ import com.google.protobuf.Message;
import com.google.protobuf.Message.Builder;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
// Uses Writables doing sasl
/**
* An RPC server that hosts protobuf described Services.
@ -254,7 +253,7 @@ public class RpcServer implements RpcServerInterface {
private final int warnResponseTime;
private final int warnResponseSize;
private final Object serverInstance;
private final Server server;
private final List<BlockingServiceAndInterface> services;
private final RpcScheduler scheduler;
@ -318,6 +317,7 @@ public class RpcServer implements RpcServerInterface {
* Short string representation without param info because param itself could be huge depends on
* the payload of a command
*/
@SuppressWarnings("deprecation")
String toShortString() {
String serviceName = this.connection.service != null?
this.connection.service.getDescriptorForType().getName() : "null";
@ -1837,7 +1837,7 @@ public class RpcServer implements RpcServerInterface {
/**
* Constructs a server listening on the named port and address.
* @param serverInstance hosting instance of {@link Server}. We will do authentications if an
* @param server hosting instance of {@link Server}. We will do authentications if an
* instance else pass null for no authentication check.
* @param name Used keying this rpc servers' metrics and for naming the Listener thread.
* @param services A list of services.
@ -1845,12 +1845,12 @@ public class RpcServer implements RpcServerInterface {
* @param conf
* @throws IOException
*/
public RpcServer(final Server serverInstance, final String name,
public RpcServer(final Server server, final String name,
final List<BlockingServiceAndInterface> services,
final InetSocketAddress isa, Configuration conf,
RpcScheduler scheduler)
throws IOException {
this.serverInstance = serverInstance;
this.server = server;
this.services = services;
this.isa = isa;
this.conf = conf;
@ -1933,32 +1933,15 @@ public class RpcServer implements RpcServerInterface {
@Override
public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
/** Starts the service. Must be called before any calls will be handled. */
@Override
public void start() {
startThreads();
openServer();
}
/**
* Open a previously started server.
*/
@Override
public void openServer() {
this.started = true;
}
@Override
public boolean isStarted() {
return this.started;
}
/**
* Starts the service threads but does not allow requests to be responded yet.
* Client will get {@link ServerNotRunningYetException} instead.
*/
/** Starts the service. Must be called before any calls will be handled. */
@Override
public synchronized void startThreads() {
public synchronized void start() {
if (started) return;
AuthenticationTokenSecretManager mgr = createSecretManager();
if (mgr != null) {
setSecretManager(mgr);
@ -1969,6 +1952,7 @@ public class RpcServer implements RpcServerInterface {
responder.start();
listener.start();
scheduler.start();
started = true;
}
@Override
@ -1980,9 +1964,7 @@ public class RpcServer implements RpcServerInterface {
private AuthenticationTokenSecretManager createSecretManager() {
if (!isSecurityEnabled) return null;
if (serverInstance == null) return null;
if (!(serverInstance instanceof org.apache.hadoop.hbase.Server)) return null;
org.apache.hadoop.hbase.Server server = (org.apache.hadoop.hbase.Server)serverInstance;
if (server == null) return null;
Configuration conf = server.getConfiguration();
long keyUpdateInterval =
conf.getLong("hbase.auth.key.update.interval", 24*60*60*1000);
@ -2084,9 +2066,9 @@ public class RpcServer implements RpcServerInterface {
responseInfo.put("queuetimems", qTime);
responseInfo.put("responsesize", responseSize);
responseInfo.put("client", clientAddress);
responseInfo.put("class", serverInstance == null? "": serverInstance.getClass().getSimpleName());
responseInfo.put("class", server == null? "": server.getClass().getSimpleName());
responseInfo.put("method", methodName);
if (params.length == 2 && serverInstance instanceof HRegionServer &&
if (params.length == 2 && server instanceof HRegionServer &&
params[0] instanceof byte[] &&
params[1] instanceof Operation) {
// if the slow process is a query, we want to log its table as well
@ -2099,7 +2081,7 @@ public class RpcServer implements RpcServerInterface {
// report to the log file
LOG.warn("(operation" + tag + "): " +
MAPPER.writeValueAsString(responseInfo));
} else if (params.length == 1 && serverInstance instanceof HRegionServer &&
} else if (params.length == 1 && server instanceof HRegionServer &&
params[0] instanceof Operation) {
// annotate the response map with operation details
responseInfo.putAll(((Operation) params[0]).toMap());
@ -2181,7 +2163,6 @@ public class RpcServer implements RpcServerInterface {
* @param addr InetAddress of incoming connection
* @throws org.apache.hadoop.security.authorize.AuthorizationException when the client isn't authorized to talk the protocol
*/
@SuppressWarnings("static-access")
public void authorize(UserGroupInformation user, ConnectionHeader connection, InetAddress addr)
throws AuthorizationException {
if (authorize) {

View File

@ -44,8 +44,6 @@ import com.google.protobuf.ServiceException;
@InterfaceAudience.Private
public interface RpcServerInterface {
void start();
void openServer();
void startThreads();
boolean isStarted();
void stop();

View File

@ -30,8 +30,8 @@ import com.google.common.base.Strings;
import com.google.common.collect.Lists;
/**
* A scheduler that maintains isolated handler pools for general, high-priority and replication
* requests.
* A scheduler that maintains isolated handler pools for general,
* high-priority, and replication requests.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving

View File

@ -24,10 +24,10 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.ZNodeClearer;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZNodeClearer;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -65,6 +65,7 @@ public class ActiveMasterManager extends ZooKeeperListener {
*/
ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master) {
super(watcher);
watcher.registerListener(this);
this.sn = sn;
this.master = master;
}
@ -139,13 +140,15 @@ public class ActiveMasterManager extends ZooKeeperListener {
*
* This also makes sure that we are watching the master znode so will be
* notified if another master dies.
* @param startupStatus
* @param checkInterval the interval to check if the master is stopped
* @param startupStatus the monitor status to track the progress
* @return True if no issue becoming active master else false if another
* master was running or if some other problem (zookeeper, stop flag has been
* set on this Master)
*/
boolean blockUntilBecomingActiveMaster(MonitoredTask startupStatus) {
while (true) {
boolean blockUntilBecomingActiveMaster(
int checkInterval, MonitoredTask startupStatus) {
while (!(master.isAborted() || master.isStopped())) {
startupStatus.setStatus("Trying to register in ZK as active master");
// Try to become the active master, watch if there is another master.
// Write out our ServerName as versioned bytes.
@ -222,9 +225,9 @@ public class ActiveMasterManager extends ZooKeeperListener {
return false;
}
synchronized (this.clusterHasActiveMaster) {
while (this.clusterHasActiveMaster.get() && !this.master.isStopped()) {
while (clusterHasActiveMaster.get() && !master.isStopped()) {
try {
this.clusterHasActiveMaster.wait();
clusterHasActiveMaster.wait(checkInterval);
} catch (InterruptedException e) {
// We expect to be interrupted when a master dies,
// will fall out if so
@ -235,18 +238,15 @@ public class ActiveMasterManager extends ZooKeeperListener {
this.master.stop(
"Cluster went down before this master became active");
}
if (this.master.isStopped()) {
return false;
}
// there is no active master so we can try to become active master again
}
}
return false;
}
/**
* @return True if cluster has an active master.
*/
public boolean isActiveMaster() {
boolean hasActiveMaster() {
try {
if (ZKUtil.checkExists(watcher, watcher.getMasterAddressZNode()) >= 0) {
return true;
@ -261,6 +261,11 @@ public class ActiveMasterManager extends ZooKeeperListener {
public void stop() {
try {
synchronized (clusterHasActiveMaster) {
// Master is already stopped, wake up the manager
// thread so that it can shutdown soon.
clusterHasActiveMaster.notifyAll();
}
// If our address is in ZK, delete it on our way out
ServerName activeMaster = null;
try {

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.monitoring.LogMonitoring;
import org.apache.hadoop.hbase.monitoring.StateDumpServlet;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.regionserver.RSDumpServlet;
import org.apache.hadoop.util.ReflectionUtils;
@InterfaceAudience.Private
@ -94,7 +95,12 @@ public class MasterDumpServlet extends StateDumpServlet {
out.println(LINE);
long tailKb = getTailKbParam(request);
LogMonitoring.dumpTailOfLogs(out, tailKb);
out.println("\n\nRS Queue:");
out.println(LINE);
if(isShowQueueDump(conf)) {
RSDumpServlet.dumpQueue(master, out);
}
out.flush();
}

View File

@ -105,7 +105,7 @@ public class MasterFileSystem {
}
};
public MasterFileSystem(Server master, MasterServices services, boolean masterRecovery)
public MasterFileSystem(Server master, MasterServices services)
throws IOException {
this.conf = master.getConfiguration();
this.master = master;
@ -129,7 +129,7 @@ public class MasterFileSystem {
HFileSystem.addLocationsOrderInterceptor(conf);
this.splitLogManager = new SplitLogManager(master.getZooKeeper(),
master.getConfiguration(), master, services,
master.getServerName(), masterRecovery);
master.getServerName());
}
/**

File diff suppressed because it is too large Load Diff

View File

@ -22,13 +22,13 @@ import java.io.IOException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.executor.ExecutorService;
@ -68,7 +68,7 @@ public interface MasterServices extends Server {
/**
* @return Master's instance of {@link MasterCoprocessorHost}
*/
MasterCoprocessorHost getCoprocessorHost();
MasterCoprocessorHost getMasterCoprocessorHost();
/**
* Check table is modifiable; i.e. exists and is offline.

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -34,9 +33,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
import org.apache.hadoop.hbase.util.FSUtils;
import com.google.protobuf.ServiceException;
/**
@ -55,6 +55,14 @@ public class MasterStatusServlet extends HttpServlet {
HMaster master = (HMaster) getServletContext().getAttribute(HMaster.MASTER);
assert master != null : "No Master in context!";
response.setContentType("text/html");
if (!master.isOnline()) {
response.getWriter().write("The Master is initializing!");
response.getWriter().close();
return;
}
Configuration conf = master.getConfiguration();
HBaseAdmin admin = new HBaseAdmin(conf);
@ -73,8 +81,7 @@ public class MasterStatusServlet extends HttpServlet {
servers = master.getServerManager().getOnlineServersList();
deadServers = master.getServerManager().getDeadServers().copyServerNames();
}
response.setContentType("text/html");
MasterStatusTmpl tmpl;
try {
tmpl = new MasterStatusTmpl()
@ -82,8 +89,8 @@ public class MasterStatusServlet extends HttpServlet {
.setMetaLocation(metaLocation)
.setServers(servers)
.setDeadServers(deadServers)
.setCatalogJanitorEnabled(master.isCatalogJanitorEnabled(null,
RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue());
.setCatalogJanitorEnabled(master.getMasterRpcServices().isCatalogJanitorEnabled(
null, RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue());
} catch (ServiceException s) {
admin.close();
throw new IOException(s);

View File

@ -46,7 +46,7 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
@Override
public String getZookeeperQuorum() {
ZooKeeperWatcher zk = master.getZooKeeperWatcher();
ZooKeeperWatcher zk = master.getZooKeeper();
if (zk == null) {
return "";
}
@ -55,7 +55,7 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
@Override
public String[] getCoprocessors() {
return master.getCoprocessors();
return master.getMasterCoprocessors();
}
@Override

View File

@ -671,6 +671,16 @@ public class RegionStates {
numServers++;
}
}
if (numServers > 1) {
// The master region server holds only a couple regions.
// Don't consider this server in calculating the average load
// if there are other region servers to avoid possible confusion.
Set<HRegionInfo> hris = serverHoldings.get(server.getServerName());
if (hris != null) {
totalLoad -= hris.size();
numServers--;
}
}
return numServers == 0 ? 0.0 :
(double)totalLoad / (double)numServers;
}

View File

@ -58,9 +58,13 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Triple;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.ServiceException;
@ -182,7 +186,6 @@ public class ServerManager {
this(master, services, true);
}
@SuppressWarnings("deprecation")
ServerManager(final Server master, final MasterServices services,
final boolean connect) throws IOException {
this.master = master;
@ -441,12 +444,21 @@ public class ServerManager {
void letRegionServersShutdown() {
long previousLogTime = 0;
ServerName sn = master.getServerName();
ZooKeeperWatcher zkw = master.getZooKeeper();
while (!onlineServers.isEmpty()) {
if (System.currentTimeMillis() > (previousLogTime + 1000)) {
Set<ServerName> remainingServers = onlineServers.keySet();
synchronized (onlineServers) {
if (remainingServers.size() == 1 && remainingServers.contains(sn)) {
// Master will delete itself later.
return;
}
}
StringBuilder sb = new StringBuilder();
// It's ok here to not sync on onlineServers - merely logging
for (ServerName key : this.onlineServers.keySet()) {
for (ServerName key : remainingServers) {
if (sb.length() > 0) {
sb.append(", ");
}
@ -456,6 +468,19 @@ public class ServerManager {
previousLogTime = System.currentTimeMillis();
}
try {
List<String> servers = ZKUtil.listChildrenNoWatch(zkw, zkw.rsZNode);
if (servers == null || (servers.size() == 1
&& servers.contains(sn.toString()))) {
LOG.info("ZK shows there is only the master self online, exiting now");
// Master could have lost some ZK events, no need to wait more.
break;
}
} catch (KeeperException ke) {
LOG.warn("Failed to list regionservers", ke);
// ZK is malfunctioning, don't hang here
break;
}
synchronized (onlineServers) {
try {
onlineServers.wait(100);
@ -471,6 +496,12 @@ public class ServerManager {
* shutdown processing.
*/
public synchronized void expireServer(final ServerName serverName) {
if (serverName.equals(master.getServerName())) {
if (!(master.isAborted() || master.isStopped())) {
master.stop("We lost our znode?");
}
return;
}
if (!services.isServerShutdownHandlerEnabled()) {
LOG.info("Master doesn't enable ServerShutdownHandler during initialization, "
+ "delay expiring server " + serverName);
@ -758,12 +789,18 @@ public class ServerManager {
* @throws IOException
* @throws RetriesExhaustedException wrapping a ConnectException if failed
*/
@SuppressWarnings("deprecation")
private AdminService.BlockingInterface getRsAdmin(final ServerName sn)
throws IOException {
AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
if (admin == null) {
LOG.debug("New admin connection to " + sn.toString());
admin = this.connection.getAdmin(sn);
if (sn.equals(master.getServerName()) && master instanceof HRegionServer) {
// A master is also a region server now, see HBASE-10569 for details
admin = ((HRegionServer)master).getRSRpcServices();
} else {
admin = this.connection.getAdmin(sn);
}
this.rsAdmins.put(sn, admin);
}
return admin;
@ -813,12 +850,10 @@ public class ServerManager {
long lastCountChange = startTime;
int count = countOfRegionServers();
int oldCount = 0;
while (
!this.master.isStopped() &&
count < maxToStart &&
(lastCountChange+interval > now || timeout > slept || count < minToStart)
){
ServerName masterSn = master.getServerName();
boolean selfCheckedIn = isServerOnline(masterSn);
while (!this.master.isStopped() && !selfCheckedIn && count < maxToStart
&& (lastCountChange+interval > now || timeout > slept || count < minToStart)) {
// Log some info at every interval time or if there is a change
if (oldCount != count || lastLogTime+interval < now){
lastLogTime = now;
@ -837,6 +872,8 @@ public class ServerManager {
now = System.currentTimeMillis();
slept = now - startTime;
selfCheckedIn = isServerOnline(masterSn);
oldCount = count;
count = countOfRegionServers();
if (count != oldCount) {
@ -942,7 +979,6 @@ public class ServerManager {
// Remove the deadNotExpired servers from the server list.
removeDeadNotExpiredServers(destServers);
return destServers;
}

View File

@ -46,7 +46,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters;
@ -151,25 +150,7 @@ public class SplitLogManager extends ZooKeeperListener {
/**
* Wrapper around {@link #SplitLogManager(ZooKeeperWatcher zkw, Configuration conf,
* Stoppable stopper, MasterServices master, ServerName serverName,
* boolean masterRecovery, TaskFinisher tf)}
* with masterRecovery = false, and tf = null. Used in unit tests.
*
* @param zkw the ZK watcher
* @param conf the HBase configuration
* @param stopper the stoppable in case anything is wrong
* @param master the master services
* @param serverName the master server name
*/
public SplitLogManager(ZooKeeperWatcher zkw, final Configuration conf,
Stoppable stopper, MasterServices master, ServerName serverName) {
this(zkw, conf, stopper, master, serverName, false, null);
}
/**
* Wrapper around {@link #SplitLogManager(ZooKeeperWatcher zkw, Configuration conf,
* Stoppable stopper, MasterServices master, ServerName serverName,
* boolean masterRecovery, TaskFinisher tf)}
* Stoppable stopper, MasterServices master, ServerName serverName, TaskFinisher tf)}
* that provides a task finisher for copying recovered edits to their final destination.
* The task finisher has to be robust because it can be arbitrarily restarted or called
* multiple times.
@ -179,11 +160,10 @@ public class SplitLogManager extends ZooKeeperListener {
* @param stopper the stoppable in case anything is wrong
* @param master the master services
* @param serverName the master server name
* @param masterRecovery an indication if the master is in recovery
*/
public SplitLogManager(ZooKeeperWatcher zkw, final Configuration conf,
Stoppable stopper, MasterServices master, ServerName serverName, boolean masterRecovery) {
this(zkw, conf, stopper, master, serverName, masterRecovery, new TaskFinisher() {
Stoppable stopper, MasterServices master, ServerName serverName) {
this(zkw, conf, stopper, master, serverName, new TaskFinisher() {
@Override
public Status finish(ServerName workerName, String logfile) {
try {
@ -207,12 +187,11 @@ public class SplitLogManager extends ZooKeeperListener {
* @param stopper the stoppable in case anything is wrong
* @param master the master services
* @param serverName the master server name
* @param masterRecovery an indication if the master is in recovery
* @param tf task finisher
*/
public SplitLogManager(ZooKeeperWatcher zkw, Configuration conf,
Stoppable stopper, MasterServices master,
ServerName serverName, boolean masterRecovery, TaskFinisher tf) {
ServerName serverName, TaskFinisher tf) {
super(zkw);
this.taskFinisher = tf;
this.conf = conf;
@ -233,10 +212,8 @@ public class SplitLogManager extends ZooKeeperListener {
this.failedDeletions = Collections.synchronizedSet(new HashSet<String>());
if (!masterRecovery) {
Threads.setDaemonThreadRunning(timeoutMonitor.getThread(), serverName
+ ".splitLogManagerTimeoutMonitor");
}
Threads.setDaemonThreadRunning(timeoutMonitor.getThread(), serverName
+ ".splitLogManagerTimeoutMonitor");
// Watcher can be null during tests with Mock'd servers.
if (this.watcher != null) {
this.watcher.registerListener(this);

View File

@ -22,6 +22,8 @@ import java.util.Arrays;
import java.util.Comparator;
import java.util.Deque;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@ -38,9 +40,12 @@ import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
import com.google.common.base.Joiner;
import com.google.common.collect.ArrayListMultimap;
@ -56,12 +61,32 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
private static final int MIN_SERVER_BALANCE = 2;
private volatile boolean stopped = false;
protected static final Set<String> TABLES_ON_MASTER = new HashSet<String>();
/**
* Regions of these tables will be put on the master regionserver by default.
*/
static {
TABLES_ON_MASTER.add(AccessControlLists.ACL_TABLE_NAME.getNameAsString());
TABLES_ON_MASTER.add(TableName.NAMESPACE_TABLE_NAME.getNameAsString());
TABLES_ON_MASTER.add(TableName.META_TABLE_NAME.getNameAsString());
}
/**
* Check if a region belongs to some small system table.
* If so, it may be expected to be put on the master regionserver.
*/
protected static boolean shouldBeOnMaster(HRegionInfo region) {
return TABLES_ON_MASTER.contains(region.getTable().getNameAsString());
}
/**
* An efficient array based implementation similar to ClusterState for keeping
* the status of the cluster in terms of region assignment and distribution.
* To be used by LoadBalancers.
*/
protected static class Cluster {
ServerName masterServerName;
ServerName[] servers;
ArrayList<String> tables;
HRegionInfo[] regions;
@ -74,6 +99,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
int[] regionIndexToTableIndex; //regionIndex -> tableIndex
int[][] numRegionsPerServerPerTable; //serverIndex -> tableIndex -> # regions
int[] numMaxRegionsPerTable; //tableIndex -> max number of regions in a single RS
int numUserRegionsOnMaster; //number of user regions on the active master
Integer[] serverIndicesSortedByRegionCount;
@ -87,9 +113,13 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
int numMovedRegions = 0; //num moved regions from the initial configuration
int numMovedMetaRegions = 0; //num of moved regions that are META
protected Cluster(Map<ServerName, List<HRegionInfo>> clusterState, Map<String, Deque<RegionLoad>> loads,
@SuppressWarnings("unchecked")
protected Cluster(ServerName masterServerName,
Map<ServerName, List<HRegionInfo>> clusterState,
Map<String, Deque<RegionLoad>> loads,
RegionLocationFinder regionFinder) {
this.masterServerName = masterServerName;
serversToIndex = new HashMap<String, Integer>();
tablesToIndex = new HashMap<String, Integer>();
//regionsToIndex = new HashMap<HRegionInfo, Integer>();
@ -147,6 +177,14 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
regionsPerServer[serverIndex] = new int[entry.getValue().size()];
}
serverIndicesSortedByRegionCount[serverIndex] = serverIndex;
if (servers[serverIndex].equals(masterServerName)) {
for (HRegionInfo hri: entry.getValue()) {
if (!shouldBeOnMaster(hri)) {
numUserRegionsOnMaster++;
}
}
}
}
for (Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
@ -218,6 +256,21 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
public void moveOrSwapRegion(int lServer, int rServer, int lRegion, int rRegion) {
if (servers[lServer].equals(masterServerName)) {
if (lRegion >= 0 && !shouldBeOnMaster(regions[lRegion])) {
numUserRegionsOnMaster--;
}
if (rRegion >= 0 && !shouldBeOnMaster(regions[rRegion])) {
numUserRegionsOnMaster++;
}
} else if (servers[rServer].equals(masterServerName)) {
if (lRegion >= 0 && !shouldBeOnMaster(regions[lRegion])) {
numUserRegionsOnMaster++;
}
if (rRegion >= 0 && !shouldBeOnMaster(regions[rRegion])) {
numUserRegionsOnMaster--;
}
}
//swap
if (rRegion >= 0 && lRegion >= 0) {
regionMoved(rRegion, rServer, lServer);
@ -354,6 +407,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
private static final Log LOG = LogFactory.getLog(BaseLoadBalancer.class);
protected final MetricsBalancer metricsBalancer = new MetricsBalancer();
protected ServerName masterServerName;
protected MasterServices services;
@Override
@ -369,6 +423,52 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
this.slop = conf.getFloat("hbase.regions.slop", (float) 0.2);
}
/**
* Balance the regions that should be on master regionserver.
*/
protected List<RegionPlan> balanceMasterRegions(
Map<ServerName, List<HRegionInfo>> clusterMap) {
if (services == null || clusterMap.size() <= 1) return null;
List<RegionPlan> plans = null;
List<HRegionInfo> regions = clusterMap.get(masterServerName);
if (regions != null) {
Iterator<ServerName> keyIt = null;
for (HRegionInfo region: regions) {
if (shouldBeOnMaster(region)) continue;
// Find a non-master regionserver to host the region
if (keyIt == null || !keyIt.hasNext()) {
keyIt = clusterMap.keySet().iterator();
}
ServerName dest = keyIt.next();
if (masterServerName.equals(dest)) {
dest = keyIt.next();
}
// Move this region away from the master regionserver
RegionPlan plan = new RegionPlan(region, masterServerName, dest);
if (plans == null) {
plans = new ArrayList<RegionPlan>();
}
plans.add(plan);
}
}
for (Map.Entry<ServerName, List<HRegionInfo>> server: clusterMap.entrySet()) {
if (masterServerName.equals(server.getKey())) continue;
for (HRegionInfo region: server.getValue()) {
if (!shouldBeOnMaster(region)) continue;
// Move this region to the master regionserver
RegionPlan plan = new RegionPlan(region, server.getKey(), masterServerName);
if (plans == null) {
plans = new ArrayList<RegionPlan>();
}
plans.add(plan);
}
}
return plans;
}
@Override
public Configuration getConf() {
return this.config;
@ -381,6 +481,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
@Override
public void setMasterServices(MasterServices masterServices) {
masterServerName = masterServices.getServerName();
this.services = masterServices;
}
@ -438,19 +539,43 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
return null;
}
Map<ServerName, List<HRegionInfo>> assignments = new TreeMap<ServerName, List<HRegionInfo>>();
int numRegions = regions.size();
int numServers = servers.size();
int max = (int) Math.ceil((float) numRegions / numServers);
int serverIdx = 0;
if (numServers > 1) {
serverIdx = RANDOM.nextInt(numServers);
if (numServers == 1) { // Only one server, nothing fancy we can do here
assignments.put(servers.get(0), new ArrayList<HRegionInfo>(regions));
return assignments;
}
int numRegions = regions.size();
// Master regionserver is in the server list.
boolean masterIncluded = servers.contains(masterServerName);
int skipServers = numServers;
if (masterIncluded) {
skipServers--;
}
int max = (int) Math.ceil((float) numRegions / skipServers);
int serverIdx = RANDOM.nextInt(numServers);
int regionIdx = 0;
for (int j = 0; j < numServers; j++) {
ServerName server = servers.get((j + serverIdx) % numServers);
if (server.equals(masterServerName)) {
// Don't put non-special region on the master regionserver,
// So that it is not overloaded.
continue;
}
List<HRegionInfo> serverRegions = new ArrayList<HRegionInfo>(max);
for (int i = regionIdx; i < numRegions; i += numServers) {
serverRegions.add(regions.get(i % numRegions));
for (int i = regionIdx; i < numRegions; i += skipServers) {
HRegionInfo region = regions.get(i % numRegions);
if (!(masterIncluded && shouldBeOnMaster(region))) {
serverRegions.add(region);
continue;
}
// Master is in the target list and this is a special region
List<HRegionInfo> masterRegions = assignments.get(masterServerName);
if (masterRegions == null) {
masterRegions = new ArrayList<HRegionInfo>(max);
assignments.put(masterServerName, masterRegions);
}
masterRegions.add(region);
}
assignments.put(server, serverRegions);
regionIdx++;
@ -498,7 +623,18 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
LOG.warn("Wanted to do random assignment but no servers to assign to");
return null;
}
return servers.get(RANDOM.nextInt(servers.size()));
int numServers = servers.size();
if (numServers == 1) return servers.get(0);
if (shouldBeOnMaster(regionInfo) && servers.contains(masterServerName)) {
return masterServerName;
}
int i = RANDOM.nextInt(numServers);
ServerName sn = servers.get(i);
if (sn.equals(masterServerName)) {
i = (i == 0 ? 1 : i - 1);
sn = servers.get(i);
}
return sn;
}
/**
@ -524,6 +660,16 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
// Update metrics
metricsBalancer.incrMiscInvocations();
if (regions.isEmpty() || servers.isEmpty()) {
return null;
}
Map<ServerName, List<HRegionInfo>> assignments = new TreeMap<ServerName, List<HRegionInfo>>();
int numServers = servers.size();
if (numServers == 1) { // Only one server, nothing fancy we can do here
assignments.put(servers.get(0), new ArrayList<HRegionInfo>(regions.keySet()));
return assignments;
}
// Group all of the old assignments by their hostname.
// We can't group directly by ServerName since the servers all have
// new start-codes.
@ -532,12 +678,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
// servers on the same host on different ports.
ArrayListMultimap<String, ServerName> serversByHostname = ArrayListMultimap.create();
for (ServerName server : servers) {
serversByHostname.put(server.getHostname(), server);
if (!server.equals(masterServerName)) {
serversByHostname.put(server.getHostname(), server);
}
}
// Now come up with new assignments
Map<ServerName, List<HRegionInfo>> assignments = new TreeMap<ServerName, List<HRegionInfo>>();
for (ServerName server : servers) {
assignments.put(server, new ArrayList<HRegionInfo>());
}
@ -547,6 +692,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
// after the cluster restart.
Set<String> oldHostsNoLongerPresent = Sets.newTreeSet();
// Master regionserver is in the server list.
boolean masterIncluded = servers.contains(masterServerName);
int numRandomAssignments = 0;
int numRetainedAssigments = 0;
for (Map.Entry<HRegionInfo, ServerName> entry : regions.entrySet()) {
@ -556,10 +704,22 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
if (oldServerName != null) {
localServers = serversByHostname.get(oldServerName.getHostname());
}
if (localServers.isEmpty()) {
if (masterIncluded && shouldBeOnMaster(region)) {
assignments.get(masterServerName).add(region);
if (localServers.contains(masterServerName)) {
numRetainedAssigments++;
} else {
numRandomAssignments++;
}
} else if (localServers.isEmpty()) {
// No servers on the new cluster match up with this hostname,
// assign randomly.
ServerName randomServer = servers.get(RANDOM.nextInt(servers.size()));
int i = RANDOM.nextInt(numServers);
ServerName randomServer = servers.get(i);
if (randomServer.equals(masterServerName)) {
i = (i == 0 ? 1 : i - 1);
randomServer = servers.get(i);
}
assignments.get(randomServer).add(region);
numRandomAssignments++;
if (oldServerName != null) oldHostsNoLongerPresent.add(oldServerName.getHostname());

View File

@ -35,20 +35,29 @@ public class ClusterLoadState {
private int numRegions = 0;
private int numServers = 0;
public ClusterLoadState(Map<ServerName, List<HRegionInfo>> clusterState) {
super();
public ClusterLoadState(ServerName master,
Map<ServerName, List<HRegionInfo>> clusterState) {
this.numRegions = 0;
this.numServers = clusterState.size();
this.clusterState = clusterState;
serversByLoad = new TreeMap<ServerAndLoad, List<HRegionInfo>>();
// Iterate so we can count regions as we build the map
for (Map.Entry<ServerName, List<HRegionInfo>> server : clusterState.entrySet()) {
if (master != null && numServers > 1 && master.equals(server.getKey())) {
// Don't count the master regionserver since its
// load is meant to be low.
continue;
}
List<HRegionInfo> regions = server.getValue();
int sz = regions.size();
if (sz == 0) emptyRegionServerPresent = true;
numRegions += sz;
serversByLoad.put(new ServerAndLoad(server.getKey(), sz), regions);
}
if (master != null && numServers > 1
&& clusterState.containsKey(master)) {
numServers--;
}
}
Map<ServerName, List<HRegionInfo>> getClusterState() {

View File

@ -180,10 +180,14 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
*/
public List<RegionPlan> balanceCluster(
Map<ServerName, List<HRegionInfo>> clusterMap) {
List<RegionPlan> regionsToReturn = balanceMasterRegions(clusterMap);
if (regionsToReturn != null) {
return regionsToReturn;
}
boolean emptyRegionServerPresent = false;
long startTime = System.currentTimeMillis();
ClusterLoadState cs = new ClusterLoadState(clusterMap);
ClusterLoadState cs = new ClusterLoadState(masterServerName, clusterMap);
if (!this.needsBalance(cs)) return null;
@ -204,7 +208,7 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
// TODO: Look at data block locality or a more complex load to do this
MinMaxPriorityQueue<RegionPlan> regionsToMove =
MinMaxPriorityQueue.orderedBy(rpComparator).create();
List<RegionPlan> regionsToReturn = new ArrayList<RegionPlan>();
regionsToReturn = new ArrayList<RegionPlan>();
// Walk down most loaded, pruning each to the max
int serversOverloaded = 0;
@ -233,8 +237,9 @@ public class SimpleLoadBalancer extends BaseLoadBalancer {
hri = regions.get(regions.size() - 1 - i);
}
i++;
// Don't rebalance meta regions.
if (hri.isMetaRegion()) continue;
// Don't rebalance special regions.
if (shouldBeOnMaster(hri)
&& masterServerName.equals(sal.getServerName())) continue;
regionsToMove.add(new RegionPlan(hri, sal.getServerName(), null));
numTaken++;
if (numTaken >= numToOffload) break;

View File

@ -150,6 +150,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
costFunctions = new CostFunction[]{
new RegionCountSkewCostFunction(conf),
new RegionOnMasterCostFunction(conf),
new MoveCostFunction(conf),
localityCost,
new TableSkewCostFunction(conf),
@ -191,14 +192,18 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
*/
@Override
public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState) {
if (!needsBalance(new ClusterLoadState(clusterState))) {
List<RegionPlan> plans = balanceMasterRegions(clusterState);
if (plans != null) {
return plans;
}
if (!needsBalance(new ClusterLoadState(masterServerName, clusterState))) {
return null;
}
long startTime = EnvironmentEdgeManager.currentTimeMillis();
// Keep track of servers to iterate through them.
Cluster cluster = new Cluster(clusterState, loads, regionFinder);
Cluster cluster = new Cluster(masterServerName, clusterState, loads, regionFinder);
double currentCost = computeCost(cluster, Double.MAX_VALUE);
double initCost = currentCost;
@ -257,7 +262,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
metricsBalancer.balanceCluster(endTime - startTime);
if (initCost > currentCost) {
List<RegionPlan> plans = createRegionPlans(cluster);
plans = createRegionPlans(cluster);
if (LOG.isDebugEnabled()) {
LOG.debug("Finished computing new load balance plan. Computation took "
+ (endTime - startTime) + "ms to try " + step
@ -563,8 +568,8 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
*/
public abstract static class CostFunction {
private float multiplier = 0;
private Configuration conf;
protected float multiplier = 0;
protected Configuration conf;
CostFunction(Configuration c) {
this.conf = c;
@ -742,6 +747,29 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
}
}
/**
* Compute the cost of a potential cluster configuration based upon if putting
* user regions on the master regionserver.
*/
public static class RegionOnMasterCostFunction extends CostFunction {
private static final String REGION_ON_MASTER_COST_KEY =
"hbase.master.balancer.stochastic.regionOnMasterCost";
private static final float DEFAULT_REGION_ON_MASTER__COST = 1000;
RegionOnMasterCostFunction(Configuration conf) {
super(conf);
this.setMultiplier(conf.getFloat(
REGION_ON_MASTER_COST_KEY, DEFAULT_REGION_ON_MASTER__COST));
}
@Override
double cost(Cluster cluster) {
double max = cluster.numRegions;
double value = cluster.numUserRegionsOnMaster;
return scale(0, max, value);
}
}
/**
* Compute a cost of a potential cluster configuration based upon where

View File

@ -148,7 +148,7 @@ public class CreateTableHandler extends EventHandler {
LOG.info("Create table " + tableName);
try {
MasterCoprocessorHost cpHost = ((HMaster) this.server).getCoprocessorHost();
MasterCoprocessorHost cpHost = ((HMaster) this.server).getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preCreateTableHandler(this.hTableDescriptor, this.newRegions);
}

View File

@ -61,7 +61,7 @@ public class DeleteTableHandler extends TableEventHandler {
protected void handleTableOperation(List<HRegionInfo> regions)
throws IOException, KeeperException {
MasterCoprocessorHost cpHost = ((HMaster) this.server)
.getCoprocessorHost();
.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preDeleteTableHandler(this.tableName);
}

View File

@ -129,7 +129,7 @@ public class DisableTableHandler extends EventHandler {
try {
LOG.info("Attempting to disable table " + this.tableName);
MasterCoprocessorHost cpHost = ((HMaster) this.server)
.getCoprocessorHost();
.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preDisableTableHandler(this.tableName);
}

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -48,7 +47,6 @@ import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.zookeeper.KeeperException;
import org.htrace.Trace;
/**
* Handler to run enable of a table.
@ -139,7 +137,7 @@ public class EnableTableHandler extends EventHandler {
try {
LOG.info("Attempting to enable the table " + this.tableName);
MasterCoprocessorHost cpHost = ((HMaster) this.server)
.getCoprocessorHost();
.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preEnableTableHandler(this.tableName);
}

View File

@ -60,7 +60,7 @@ public class ModifyTableHandler extends TableEventHandler {
@Override
protected void handleTableOperation(List<HRegionInfo> hris)
throws IOException {
MasterCoprocessorHost cpHost = ((HMaster) this.server).getCoprocessorHost();
MasterCoprocessorHost cpHost = ((HMaster) this.server).getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preModifyTableHandler(this.tableName, this.htd);
}

View File

@ -61,7 +61,7 @@ public class TableAddFamilyHandler extends TableEventHandler {
protected void handleTableOperation(List<HRegionInfo> hris)
throws IOException {
MasterCoprocessorHost cpHost = ((HMaster) this.server)
.getCoprocessorHost();
.getMasterCoprocessorHost();
if(cpHost != null){
cpHost.preAddColumnHandler(this.tableName, this.familyDesc);
}

View File

@ -57,7 +57,7 @@ public class TableDeleteFamilyHandler extends TableEventHandler {
@Override
protected void handleTableOperation(List<HRegionInfo> hris) throws IOException {
MasterCoprocessorHost cpHost = ((HMaster) this.server)
.getCoprocessorHost();
.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preDeleteColumnHandler(this.tableName, this.familyName);
}

View File

@ -56,7 +56,7 @@ public class TableModifyFamilyHandler extends TableEventHandler {
@Override
protected void handleTableOperation(List<HRegionInfo> regions) throws IOException {
MasterCoprocessorHost cpHost = ((HMaster) this.server)
.getCoprocessorHost();
.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preModifyColumnHandler(this.tableName, this.familyDesc);
}

View File

@ -137,7 +137,6 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
private boolean stopped;
private MasterServices master; // Needed by TableEventHandlers
private MetricsMaster metricsMaster;
private ProcedureCoordinator coordinator;
// Is snapshot feature enabled?
@ -172,7 +171,6 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
ProcedureCoordinator coordinator, ExecutorService pool)
throws IOException, UnsupportedOperationException {
this.master = master;
this.metricsMaster = metricsMaster;
this.rootDir = master.getMasterFileSystem().getRootDir();
checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());
@ -260,7 +258,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
public void deleteSnapshot(SnapshotDescription snapshot) throws SnapshotDoesNotExistException, IOException {
// call coproc pre hook
MasterCoprocessorHost cpHost = master.getCoprocessorHost();
MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preDeleteSnapshot(snapshot);
}
@ -547,7 +545,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
.build();
// call pre coproc hook
MasterCoprocessorHost cpHost = master.getCoprocessorHost();
MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preSnapshot(snapshot, desc);
}
@ -668,7 +666,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
public void restoreSnapshot(SnapshotDescription reqSnapshot) throws IOException {
FileSystem fs = master.getMasterFileSystem().getFileSystem();
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir);
MasterCoprocessorHost cpHost = master.getCoprocessorHost();
MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
// check if the snapshot exists
if (!fs.exists(snapshotDir)) {
@ -1005,7 +1003,6 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
public void initialize(MasterServices master, MetricsMaster metricsMaster) throws KeeperException,
IOException, UnsupportedOperationException {
this.master = master;
this.metricsMaster = metricsMaster;
this.rootDir = master.getMasterFileSystem().getRootDir();
checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());

View File

@ -26,6 +26,7 @@ import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorStatus;
import org.apache.hadoop.hbase.util.VersionInfo;
@ -45,6 +46,10 @@ public abstract class StateDumpServlet extends HttpServlet {
" on " + org.apache.hadoop.util.VersionInfo.getDate());
}
protected boolean isShowQueueDump(Configuration conf){
return conf.getBoolean("hbase.regionserver.servlet.show.queuedump", true);
}
protected long getTailKbParam(HttpServletRequest request) {
String param = request.getParameter("tailkb");
if (param == null) {

View File

@ -24,7 +24,6 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@ -38,7 +37,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
import org.apache.hadoop.hbase.regionserver.HRegionServer.QosPriority;
import org.apache.hadoop.hbase.regionserver.RSRpcServices.QosPriority;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.Message;
@ -72,7 +71,7 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
private final Map<String, Integer> annotatedQos;
//We need to mock the regionserver instance for some unit tests (set via
//setRegionServer method.
private HRegionServer hRegionServer;
private RSRpcServices rpcServices;
@SuppressWarnings("unchecked")
private final Class<? extends Message>[] knownArgumentClasses = new Class[]{
GetRegionInfoRequest.class,
@ -92,10 +91,9 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
private final Map<String, Map<Class<? extends Message>, Method>> methodMap =
new HashMap<String, Map<Class<? extends Message>, Method>>();
AnnotationReadingPriorityFunction(final HRegionServer hrs) {
this.hRegionServer = hrs;
AnnotationReadingPriorityFunction(final RSRpcServices rpcServices) {
Map<String, Integer> qosMap = new HashMap<String, Integer>();
for (Method m : HRegionServer.class.getMethods()) {
for (Method m : RSRpcServices.class.getMethods()) {
QosPriority p = m.getAnnotation(QosPriority.class);
if (p != null) {
// Since we protobuf'd, and then subsequently, when we went with pb style, method names
@ -107,6 +105,7 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
qosMap.put(capitalizedMethodName, p.priority());
}
}
this.rpcServices = rpcServices;
this.annotatedQos = qosMap;
if (methodMap.get("getRegion") == null) {
methodMap.put("hasRegion", new HashMap<Class<? extends Message>, Method>());
@ -129,16 +128,6 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
return strBuilder.toString();
}
public boolean isMetaRegion(byte[] regionName) {
HRegion region;
try {
region = hRegionServer.getRegion(regionName);
} catch (NotServingRegionException ignored) {
return false;
}
return region.getRegionInfo().isMetaTable();
}
@Override
public int getPriority(RequestHeader header, Message param) {
String methodName = header.getMethodName();
@ -167,7 +156,7 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
if (hasRegion != null && (Boolean)hasRegion.invoke(param, (Object[])null)) {
Method getRegion = methodMap.get("getRegion").get(rpcArgClass);
regionSpecifier = (RegionSpecifier)getRegion.invoke(param, (Object[])null);
HRegion region = hRegionServer.getRegion(regionSpecifier);
HRegion region = rpcServices.getRegion(regionSpecifier);
if (region.getRegionInfo().isMetaTable()) {
if (LOG.isTraceEnabled()) {
LOG.trace("High priority because region=" + region.getRegionNameAsString());
@ -187,7 +176,7 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
if (!request.hasScannerId()) {
return HConstants.NORMAL_QOS;
}
RegionScanner scanner = hRegionServer.getScanner(request.getScannerId());
RegionScanner scanner = rpcServices.getScanner(request.getScannerId());
if (scanner != null && scanner.getRegionInfo().isMetaRegion()) {
if (LOG.isTraceEnabled()) {
// Scanner requests are small in size so TextFormat version should not overwhelm log.
@ -201,6 +190,6 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
@VisibleForTesting
void setRegionServer(final HRegionServer hrs) {
this.hRegionServer = hrs;
this.rpcServices = hrs.getRSRpcServices();
}
}

View File

@ -59,9 +59,8 @@ public class HRegionServerCommandLine extends ServerCommandLine {
} else {
logProcessInfo(getConf());
HRegionServer hrs = HRegionServer.constructRegionServer(regionServerClass, conf);
Thread rsThread = HRegionServer.startRegionServer(hrs);
rsThread.join();
hrs.start();
hrs.join();
if (hrs.isAborted()) {
throw new RuntimeException("HRegionServer Aborted");
}

View File

@ -148,19 +148,6 @@ class LogRoller extends HasThread implements WALActionsListener {
}
}
/**
* Called by region server to wake up this thread if it sleeping.
* It is sleeping if rollLock is not held.
*/
public void interruptIfNecessary() {
try {
rollLock.lock();
this.interrupt();
} finally {
rollLock.unlock();
}
}
protected HLog getWAL() throws IOException {
return this.services.getWAL(null);
}

View File

@ -118,7 +118,7 @@ class MetricsRegionServerWrapperImpl
@Override
public String getZookeeperQuorum() {
ZooKeeperWatcher zk = regionServer.getZooKeeperWatcher();
ZooKeeperWatcher zk = regionServer.getZooKeeper();
if (zk == null) {
return "";
}
@ -127,7 +127,7 @@ class MetricsRegionServerWrapperImpl
@Override
public String getCoprocessors() {
String[] coprocessors = regionServer.getCoprocessors();
String[] coprocessors = regionServer.getRegionServerCoprocessors();
if (coprocessors == null || coprocessors.length == 0) {
return "";
}
@ -154,7 +154,7 @@ class MetricsRegionServerWrapperImpl
@Override
public long getTotalRequestCount() {
return regionServer.requestCount.get();
return regionServer.rpcServices.requestCount.get();
}
@Override

View File

@ -45,10 +45,6 @@ public class RSDumpServlet extends StateDumpServlet {
HRegionServer hrs = (HRegionServer)getServletContext().getAttribute(
HRegionServer.REGIONSERVER);
assert hrs != null : "No RS in context!";
Configuration hrsconf = (Configuration)getServletContext().getAttribute(
HRegionServer.REGIONSERVER_CONF);
assert hrsconf != null : "No RS conf in context";
response.setContentType("text/plain");
@ -61,7 +57,7 @@ public class RSDumpServlet extends StateDumpServlet {
OutputStream os = response.getOutputStream();
PrintWriter out = new PrintWriter(os);
out.println("Master status for " + hrs.getServerName()
out.println("RegionServer status for " + hrs.getServerName()
+ " as of " + new Date());
out.println("\n\nVersion Info:");
@ -94,18 +90,14 @@ public class RSDumpServlet extends StateDumpServlet {
out.println("\n\nRS Queue:");
out.println(LINE);
if(isShowQueueDump(hrsconf)) {
if(isShowQueueDump(conf)) {
dumpQueue(hrs, out);
}
out.flush();
}
private boolean isShowQueueDump(Configuration conf){
return conf.getBoolean("hbase.regionserver.servlet.show.queuedump", true);
}
private void dumpQueue(HRegionServer hrs, PrintWriter out)
public static void dumpQueue(HRegionServer hrs, PrintWriter out)
throws IOException {
// 1. Print out Compaction/Split Queue
out.println("Compaction/Split Queue summary: "
@ -117,5 +109,4 @@ public class RSDumpServlet extends StateDumpServlet {
+ hrs.cacheFlusher.toString());
out.println(hrs.cacheFlusher.dumpQueue());
}
}

View File

@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.executor.EventType;
@ -240,7 +239,8 @@ public class RegionMergeTransaction {
public HRegion execute(final Server server,
final RegionServerServices services) throws IOException {
if (rsCoprocessorHost == null) {
rsCoprocessorHost = server != null ? ((HRegionServer) server).getCoprocessorHost() : null;
rsCoprocessorHost = server != null ?
((HRegionServer) server).getRegionServerCoprocessorHost() : null;
}
HRegion mergedRegion = createMergedRegion(server, services);
if (rsCoprocessorHost != null) {

View File

@ -18,27 +18,26 @@
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.zookeeper.KeeperException;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
/**
* Services provided by {@link HRegionServer}
*/
@InterfaceAudience.Private
public interface RegionServerServices
extends OnlineRegions, FavoredNodesForRegion, PriorityFunction {
extends OnlineRegions, FavoredNodesForRegion {
/**
* @return True if this regionserver is stopping.
*/

View File

@ -18,21 +18,16 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
import org.apache.hadoop.hbase.ipc.RpcScheduler;
/**
* A factory class that constructs an {@link org.apache.hadoop.hbase.ipc.RpcScheduler} for
* a region server.
* A factory class that constructs an {@link org.apache.hadoop.hbase.ipc.RpcScheduler}.
*/
public interface RpcSchedulerFactory {
/**
* Constructs a {@link org.apache.hadoop.hbase.ipc.RpcScheduler}.
*
* Please note that this method is called in constructor of {@link HRegionServer}, so some
* fields may not be ready for access. The reason that {@code HRegionServer} is passed as
* parameter here is that an RPC scheduler may need to access data structure inside
* {@code HRegionServer} (see example in {@link SimpleRpcSchedulerFactory}).
*/
RpcScheduler create(Configuration conf, RegionServerServices server);
RpcScheduler create(Configuration conf, PriorityFunction priority);
}

View File

@ -19,14 +19,15 @@ package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
import org.apache.hadoop.hbase.ipc.RpcScheduler;
import org.apache.hadoop.hbase.ipc.SimpleRpcScheduler;
/** Constructs a {@link SimpleRpcScheduler}. for the region server. */
/** Constructs a {@link SimpleRpcScheduler}. */
class SimpleRpcSchedulerFactory implements RpcSchedulerFactory {
@Override
public RpcScheduler create(Configuration conf, RegionServerServices server) {
public RpcScheduler create(Configuration conf, PriorityFunction priority) {
int handlerCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,
HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT);
return new SimpleRpcScheduler(
@ -36,7 +37,7 @@ class SimpleRpcSchedulerFactory implements RpcSchedulerFactory {
HConstants.DEFAULT_REGION_SERVER_META_HANDLER_COUNT),
conf.getInt(HConstants.REGION_SERVER_REPLICATION_HANDLER_COUNT,
HConstants.DEFAULT_REGION_SERVER_REPLICATION_HANDLER_COUNT),
server,
priority,
HConstants.QOS_THRESHOLD);
}
}

View File

@ -29,10 +29,8 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.ShutdownHook;
import org.apache.hadoop.util.ReflectionUtils;
/**
@ -195,9 +193,6 @@ public class JVMClusterUtil {
if (regionservers != null) {
for (JVMClusterUtil.RegionServerThread t: regionservers) {
HRegionServer hrs = t.getRegionServer();
ShutdownHook.install(hrs.getConfiguration(), FileSystem.get(hrs
.getConfiguration()), hrs, t);
t.start();
}
}

View File

@ -23,7 +23,7 @@
import="org.apache.hadoop.hbase.HBaseConfiguration"
import="org.apache.hadoop.hbase.master.HMaster"%><%
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
ZooKeeperWatcher watcher = master.getZooKeeperWatcher();
ZooKeeperWatcher watcher = master.getZooKeeper();
%>
<!--[if IE]>
<!DOCTYPE html>

View File

@ -97,7 +97,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
/**
* Returns an {@link MasterAdminService.BlockingInterface} to the active master
*/
public abstract MasterService.BlockingInterface getMaster()
public abstract MasterService.BlockingInterface getMasterAdminService()
throws IOException;
/**

View File

@ -40,6 +40,7 @@ import java.util.Map;
import java.util.NavigableSet;
import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
@ -75,12 +76,14 @@ import org.apache.hadoop.hbase.mapreduce.MapreduceTestingShim;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.tool.Canary;
@ -88,6 +91,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.RegionSplitter;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.Threads;
@ -124,6 +128,7 @@ import org.apache.zookeeper.ZooKeeper.States;
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@SuppressWarnings("deprecation")
public class HBaseTestingUtility extends HBaseCommonTestingUtility {
private MiniZooKeeperCluster zkCluster = null;
@ -1928,7 +1933,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return createMultiRegions(c, table, family, regionStartKeys);
}
@SuppressWarnings("deprecation")
public int createMultiRegions(final Configuration c, final HTable table,
final byte[] columnFamily, byte [][] startKeys)
throws IOException {
@ -2740,6 +2744,32 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return user;
}
public static NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
throws IOException {
NavigableSet<String> online = new TreeSet<String>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
try {
for (HRegionInfo region :
ProtobufUtil.getOnlineRegions(rst.getRegionServer().getRSRpcServices())) {
online.add(region.getRegionNameAsString());
}
} catch (RegionServerStoppedException e) {
// That's fine.
}
}
for (MasterThread mt : cluster.getLiveMasterThreads()) {
try {
for (HRegionInfo region :
ProtobufUtil.getOnlineRegions(mt.getMaster().getRSRpcServices())) {
online.add(region.getRegionNameAsString());
}
} catch (RegionServerStoppedException e) {
// That's fine.
}
}
return online;
}
/**
* Set maxRecoveryErrorCount in DFSClient. In 0.20 pre-append its hard-coded to 5 and
* makes tests linger. Here is the exception you'll see:

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
@ -375,6 +376,14 @@ public class MiniHBaseCluster extends HBaseCluster {
return t;
}
/**
* Returns the current active master, if available.
* @return the active HMaster, null if none is active.
*/
public MasterService.BlockingInterface getMasterAdminService() {
return this.hbaseCluster.getActiveMaster().getMasterRpcServices();
}
/**
* Returns the current active master, if available.
* @return the active HMaster, null if none is active.
@ -383,6 +392,19 @@ public class MiniHBaseCluster extends HBaseCluster {
return this.hbaseCluster.getActiveMaster();
}
/**
* Returns the current active master thread, if available.
* @return the active MasterThread, null if none is active.
*/
public MasterThread getMasterThread() {
for (MasterThread mt: hbaseCluster.getLiveMasters()) {
if (mt.getMaster().isActiveMaster()) {
return mt;
}
}
return null;
}
/**
* Returns the master at the specified index, if available.
* @return the active HMaster, null if none is active.
@ -490,6 +512,7 @@ public class MiniHBaseCluster extends HBaseCluster {
* Shut down the mini HBase cluster
* @throws IOException
*/
@SuppressWarnings("deprecation")
public void shutdown() throws IOException {
if (this.hbaseCluster != null) {
this.hbaseCluster.shutdown();
@ -635,6 +658,15 @@ public class MiniHBaseCluster extends HBaseCluster {
@Override
public ServerName getServerHoldingRegion(byte[] regionName) throws IOException {
// Assume there is only one master thread which is the active master.
// If there are multiple master threads, the backup master threads
// should hold some regions. Please refer to #countServedRegions
// to see how we find out all regions.
HMaster master = getMaster();
HRegion region = master.getOnlineRegion(regionName);
if (region != null) {
return master.getServerName();
}
int index = getServerWith(regionName);
if (index < 0) {
return null;
@ -653,6 +685,9 @@ public class MiniHBaseCluster extends HBaseCluster {
for (JVMClusterUtil.RegionServerThread rst : getLiveRegionServerThreads()) {
count += rst.getRegionServer().getNumberOfOnlineRegions();
}
for (JVMClusterUtil.MasterThread mt : getLiveMasterThreads()) {
count += mt.getMaster().getNumberOfOnlineRegions();
}
return count;
}
@ -711,12 +746,12 @@ public class MiniHBaseCluster extends HBaseCluster {
@Override
public AdminService.BlockingInterface getAdminProtocol(ServerName serverName) throws IOException {
return getRegionServer(getRegionServerIndex(serverName));
return getRegionServer(getRegionServerIndex(serverName)).getRSRpcServices();
}
@Override
public ClientService.BlockingInterface getClientProtocol(ServerName serverName)
throws IOException {
return getRegionServer(getRegionServerIndex(serverName));
return getRegionServer(getRegionServerIndex(serverName)).getRSRpcServices();
}
}

View File

@ -24,7 +24,6 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentSkipListMap;
import com.google.protobuf.Message;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
@ -33,7 +32,6 @@ import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.TableLockManager.NullTableLockManager;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
import org.apache.hadoop.hbase.regionserver.FlushRequester;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -217,11 +215,6 @@ class MockRegionServerServices implements RegionServerServices {
return null;
}
@Override
public int getPriority(RPCProtos.RequestHeader header, Message param) {
return 0;
}
@Override
public ServerNonceManager getNonceManager() {
// TODO Auto-generated method stub

View File

@ -134,9 +134,8 @@ public class TestDrainingServer {
balancer, startupMasterExecutor("mockExecutorService"), null, null);
Mockito.when(master.getAssignmentManager()).thenReturn(am);
Mockito.when(master.getZooKeeperWatcher()).thenReturn(zkWatcher);
Mockito.when(master.getZooKeeper()).thenReturn(zkWatcher);
am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_A));
zkWatcher.registerListenerFirst(am);

View File

@ -81,7 +81,8 @@ public class TestGlobalMemStoreSize {
for (HRegionServer server : getOnlineRegionServers()) {
long globalMemStoreSize = 0;
for (HRegionInfo regionInfo : ProtobufUtil.getOnlineRegions(server)) {
for (HRegionInfo regionInfo :
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
globalMemStoreSize +=
server.getFromOnlineRegions(regionInfo.getEncodedName()).
getMemstoreSize().get();
@ -96,7 +97,8 @@ public class TestGlobalMemStoreSize {
LOG.info("Starting flushes on " + server.getServerName() +
", size=" + server.getRegionServerAccounting().getGlobalMemstoreSize());
for (HRegionInfo regionInfo : ProtobufUtil.getOnlineRegions(server)) {
for (HRegionInfo regionInfo :
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
HRegion r = server.getFromOnlineRegions(regionInfo.getEncodedName());
flush(r, server);
}
@ -111,7 +113,8 @@ public class TestGlobalMemStoreSize {
if (size > 0) {
// If size > 0, see if its because the meta region got edits while
// our test was running....
for (HRegionInfo regionInfo : ProtobufUtil.getOnlineRegions(server)) {
for (HRegionInfo regionInfo :
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
HRegion r = server.getFromOnlineRegions(regionInfo.getEncodedName());
long l = r.getMemstoreSize().longValue();
if (l > 0) {
@ -144,15 +147,6 @@ public class TestGlobalMemStoreSize {
server.getRegionServerAccounting().getGlobalMemstoreSize());
}
/** figure out how many regions are currently being served. */
private int getRegionCount() throws IOException {
int total = 0;
for (HRegionServer server : getOnlineRegionServers()) {
total += ProtobufUtil.getOnlineRegions(server).size();
}
return total;
}
private List<HRegionServer> getOnlineRegionServers() {
List<HRegionServer> list = new ArrayList<HRegionServer>();
for (JVMClusterUtil.RegionServerThread rst :
@ -168,12 +162,14 @@ public class TestGlobalMemStoreSize {
* Wait until all the regions are assigned.
*/
private void waitForAllRegionsAssigned() throws IOException {
while (getRegionCount() < totalRegionNum) {
LOG.debug("Waiting for there to be "+totalRegionNum+" regions, but there are " + getRegionCount() + " right now.");
while (true) {
int regionCount = HBaseTestingUtility.getAllOnlineRegions(cluster).size();
if (regionCount >= totalRegionNum) break;
LOG.debug("Waiting for there to be "+ totalRegionNum
+" regions, but there are " + regionCount + " right now.");
try {
Thread.sleep(100);
} catch (InterruptedException e) {}
}
}
}

View File

@ -37,13 +37,11 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import com.google.common.collect.Sets;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
@ -52,10 +50,7 @@ import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.io.IOException;
import java.util.Set;
import static org.junit.Assert.*;
import com.google.common.collect.Sets;
@Category(MediumTests.class)
public class TestNamespace {
@ -78,7 +73,7 @@ public class TestNamespace {
cluster = TEST_UTIL.getHBaseCluster();
master = ((MiniHBaseCluster)cluster).getMaster();
zkNamespaceManager =
new ZKNamespaceManager(master.getZooKeeperWatcher());
new ZKNamespaceManager(master.getZooKeeper());
zkNamespaceManager.start();
LOG.info("Done initializing cluster");
}

View File

@ -149,9 +149,9 @@ public class TestRegionRebalancing {
/** figure out how many regions are currently being served. */
private int getRegionCount() throws IOException {
int total = 0;
int total = 0; // Regions on master are ignored since not counted for balancing
for (HRegionServer server : getOnlineRegionServers()) {
total += ProtobufUtil.getOnlineRegions(server).size();
total += ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size();
}
return total;
}
@ -183,11 +183,13 @@ public class TestRegionRebalancing {
+ ", up border: " + avgLoadPlusSlop + "; attempt: " + i);
for (HRegionServer server : servers) {
int serverLoad = ProtobufUtil.getOnlineRegions(server).size();
int serverLoad =
ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size();
LOG.debug(server.getServerName() + " Avg: " + avg + " actual: " + serverLoad);
if (!(avg > 2.0 && serverLoad <= avgLoadPlusSlop
&& serverLoad >= avgLoadMinusSlop)) {
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(server)) {
for (HRegionInfo hri :
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
if (hri.isMetaRegion()) serverLoad--;
// LOG.debug(hri.getRegionNameAsString());
}
@ -234,7 +236,7 @@ public class TestRegionRebalancing {
* Wait until all the regions are assigned.
*/
private void waitForAllRegionsAssigned() throws IOException {
int totalRegions = HBaseTestingUtility.KEYS.length+1;
int totalRegions = HBaseTestingUtility.KEYS.length;
while (getRegionCount() < totalRegions) {
// while (!cluster.getMaster().allRegionsAssigned()) {
LOG.debug("Waiting for there to be "+ totalRegions +" regions, but there are " + getRegionCount() + " right now.");

View File

@ -106,12 +106,17 @@ public class TestZooKeeper {
*/
@Before
public void setUp() throws Exception {
TEST_UTIL.startMiniHBaseCluster(1, 2);
TEST_UTIL.startMiniHBaseCluster(2, 2);
}
@After
public void after() throws Exception {
try {
// Some regionserver could fail to delete its znode.
// So shutdown could hang. Let's kill them all instead.
TEST_UTIL.getHBaseCluster().killAll();
// Still need to clean things up
TEST_UTIL.shutdownMiniHBaseCluster();
} finally {
TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
@ -205,17 +210,14 @@ public class TestZooKeeper {
connection.close();
}
@Test (timeout = 60000)
@Test (timeout = 120000)
public void testRegionServerSessionExpired() throws Exception {
LOG.info("Starting testRegionServerSessionExpired");
int metaIndex = TEST_UTIL.getMiniHBaseCluster().getServerWithMeta();
TEST_UTIL.expireRegionServerSession(metaIndex);
TEST_UTIL.expireRegionServerSession(0);
testSanity("testRegionServerSessionExpired");
}
// @Test Disabled because seems to make no sense expiring master session
// and then trying to create table (down in testSanity); on master side
// it will fail because the master's session has expired -- St.Ack 07/24/2012
@Test(timeout = 300000)
public void testMasterSessionExpired() throws Exception {
LOG.info("Starting testMasterSessionExpired");
TEST_UTIL.expireMasterSession();
@ -227,14 +229,14 @@ public class TestZooKeeper {
* test differs from {@link #testMasterSessionExpired} because here
* the master znode will exist in ZK.
*/
@Test(timeout = 60000)
@Test(timeout = 300000)
public void testMasterZKSessionRecoveryFailure() throws Exception {
LOG.info("Starting testMasterZKSessionRecoveryFailure");
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HMaster m = cluster.getMaster();
m.abort("Test recovery from zk session expired",
new KeeperException.SessionExpiredException());
assertFalse(m.isStopped());
assertTrue(m.isStopped()); // Master doesn't recover any more
testSanity("testMasterZKSessionRecoveryFailure");
}
@ -480,13 +482,13 @@ public class TestZooKeeper {
* session. Without the HBASE-6046 fix master always tries to assign all the user regions by
* calling retainAssignment.
*/
@Test
@Test(timeout = 300000)
public void testRegionAssignmentAfterMasterRecoveryDueToZKExpiry() throws Exception {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
cluster.startRegionServer();
cluster.waitForActiveAndReadyMaster(10000);
HMaster m = cluster.getMaster();
ZooKeeperWatcher zkw = m.getZooKeeperWatcher();
ZooKeeperWatcher zkw = m.getZooKeeper();
int expectedNumOfListeners = zkw.getNumberOfListeners();
// now the cluster is up. So assign some regions.
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
@ -500,17 +502,17 @@ public class TestZooKeeper {
admin.createTable(htd, SPLIT_KEYS);
ZooKeeperWatcher zooKeeperWatcher = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
ZKAssign.blockUntilNoRIT(zooKeeperWatcher);
m.getZooKeeperWatcher().close();
m.getZooKeeper().close();
MockLoadBalancer.retainAssignCalled = false;
m.abort("Test recovery from zk session expired",
new KeeperException.SessionExpiredException());
assertFalse(m.isStopped());
assertTrue(m.isStopped()); // Master doesn't recover any more
// The recovered master should not call retainAssignment, as it is not a
// clean startup.
assertFalse("Retain assignment should not be called", MockLoadBalancer.retainAssignCalled);
// number of listeners should be same as the value before master aborted
// wait for new master is initialized
cluster.waitForActiveAndReadyMaster(10000);
cluster.waitForActiveAndReadyMaster(120000);
assertEquals(expectedNumOfListeners, zkw.getNumberOfListeners());
} finally {
admin.close();
@ -521,7 +523,7 @@ public class TestZooKeeper {
* Tests whether the logs are split when master recovers from a expired zookeeper session and an
* RS goes down.
*/
@Test(timeout = 240000)
@Test(timeout = 300000)
public void testLogSplittingAfterMasterRecoveryDueToZKExpiry() throws IOException,
KeeperException, InterruptedException {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
@ -549,10 +551,10 @@ public class TestZooKeeper {
p.add(Bytes.toBytes("col"), Bytes.toBytes("ql"), Bytes.toBytes("value" + numberOfPuts));
table.put(p);
}
m.getZooKeeperWatcher().close();
m.getZooKeeper().close();
m.abort("Test recovery from zk session expired",
new KeeperException.SessionExpiredException());
assertFalse(m.isStopped());
assertTrue(m.isStopped()); // Master doesn't recover any more
cluster.getRegionServer(0).abort("Aborting");
// Without patch for HBASE-6046 this test case will always timeout
// with patch the test case should pass.

View File

@ -25,23 +25,20 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -54,22 +51,7 @@ import org.junit.experimental.categories.Category;
public class TestMetaReaderEditor {
private static final Log LOG = LogFactory.getLog(TestMetaReaderEditor.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static ZooKeeperWatcher zkw;
private static CatalogTracker CT;
private final static Abortable ABORTABLE = new Abortable() {
private final AtomicBoolean abort = new AtomicBoolean(false);
@Override
public void abort(String why, Throwable e) {
LOG.info(why, e);
abort.set(true);
}
@Override
public boolean isAborted() {
return abort.get();
}
};
@BeforeClass public static void beforeClass() throws Exception {
UTIL.startMiniCluster(3);
@ -79,13 +61,11 @@ public class TestMetaReaderEditor {
// responsive. 1 second is default as is ten retries.
c.setLong("hbase.client.pause", 1000);
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 10);
zkw = new ZooKeeperWatcher(c, "TestMetaReaderEditor", ABORTABLE);
CT = new CatalogTracker(zkw, c, ABORTABLE);
CT = new CatalogTracker(c);
CT.start();
}
@AfterClass public static void afterClass() throws Exception {
ABORTABLE.abort("test ending", null);
CT.stop();
UTIL.shutdownMiniCluster();
}

View File

@ -98,7 +98,9 @@ public class HConnectionTestingUtility {
final ClientProtos.ClientService.BlockingInterface client,
final ServerName sn, final HRegionInfo hri)
throws IOException {
ClusterConnection c = HConnectionTestingUtility.getMockedConnection(conf);
HConnectionImplementation c = Mockito.mock(HConnectionImplementation.class);
Mockito.when(c.getConfiguration()).thenReturn(conf);
ConnectionManager.CONNECTION_INSTANCES.put(new HConnectionKey(conf), c);
Mockito.doNothing().when(c).close();
// Make it so we return a particular location when asked.
final HRegionLocation loc = new HRegionLocation(hri, sn);

View File

@ -18,7 +18,6 @@
*/
package org.apache.hadoop.hbase.client;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@ -68,7 +67,12 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.*;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.protobuf.ServiceException;
@ -540,6 +544,11 @@ public class TestAdmin {
}
regs.add(entry.getKey());
}
if (numRS >= 2) {
// Ignore the master region server,
// which contains less regions by intention.
numRS--;
}
float average = (float) expectedRegions/numRS;
int min = (int)Math.floor(average);
int max = (int)Math.ceil(average);
@ -834,7 +843,7 @@ public class TestAdmin {
* @throws Exception
* @throws IOException
*/
@Test (timeout=300000)
@Test (timeout=400000)
public void testForceSplit() throws Exception {
byte[][] familyNames = new byte[][] { Bytes.toBytes("cf") };
int[] rowCounts = new int[] { 6000 };
@ -886,7 +895,7 @@ public class TestAdmin {
* @throws Exception
* @throws IOException
*/
@Test (timeout=300000)
@Test (timeout=800000)
public void testForceSplitMultiFamily() throws Exception {
int numVersions = HColumnDescriptor.DEFAULT_VERSIONS;
@ -973,7 +982,7 @@ public class TestAdmin {
final AtomicInteger count = new AtomicInteger(0);
Thread t = new Thread("CheckForSplit") {
public void run() {
for (int i = 0; i < 20; i++) {
for (int i = 0; i < 45; i++) {
try {
sleep(1000);
} catch (InterruptedException e) {
@ -1299,7 +1308,7 @@ public class TestAdmin {
HRegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.getTable().isSystemTable()) {
info = regionInfo;
@ -1307,11 +1316,13 @@ public class TestAdmin {
.getServerName().getServerName());
}
}
boolean isInList = ProtobufUtil.getOnlineRegions(rs).contains(info);
boolean isInList = ProtobufUtil.getOnlineRegions(
rs.getRSRpcServices()).contains(info);
long timeout = System.currentTimeMillis() + 10000;
while ((System.currentTimeMillis() < timeout) && (isInList)) {
Thread.sleep(100);
isInList = ProtobufUtil.getOnlineRegions(rs).contains(info);
isInList = ProtobufUtil.getOnlineRegions(
rs.getRSRpcServices()).contains(info);
}
assertFalse("The region should not be present in online regions list.",
@ -1325,7 +1336,7 @@ public class TestAdmin {
HRegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion1")) {
@ -1339,7 +1350,7 @@ public class TestAdmin {
}
}
}
onlineRegions = ProtobufUtil.getOnlineRegions(rs);
onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
assertTrue("The region should be present in online regions list.",
onlineRegions.contains(info));
}
@ -1352,7 +1363,7 @@ public class TestAdmin {
HRegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
@ -1364,11 +1375,13 @@ public class TestAdmin {
}
}
boolean isInList = ProtobufUtil.getOnlineRegions(rs).contains(info);
boolean isInList = ProtobufUtil.getOnlineRegions(
rs.getRSRpcServices()).contains(info);
long timeout = System.currentTimeMillis() + 10000;
while ((System.currentTimeMillis() < timeout) && (isInList)) {
Thread.sleep(100);
isInList = ProtobufUtil.getOnlineRegions(rs).contains(info);
isInList = ProtobufUtil.getOnlineRegions(
rs.getRSRpcServices()).contains(info);
}
assertFalse("The region should not be present in online regions list.",
@ -1383,7 +1396,7 @@ public class TestAdmin {
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
try {
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString()
@ -1407,7 +1420,7 @@ public class TestAdmin {
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
try {
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString()
@ -1430,7 +1443,7 @@ public class TestAdmin {
HRegionInfo info = null;
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLENAME);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs);
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
for (HRegionInfo regionInfo : onlineRegions) {
if (!regionInfo.isMetaTable()) {
if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion4")) {
@ -1444,7 +1457,7 @@ public class TestAdmin {
}
}
}
onlineRegions = ProtobufUtil.getOnlineRegions(rs);
onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
assertTrue("The region should be present in online regions list.",
onlineRegions.contains(info));
}
@ -1683,7 +1696,7 @@ public class TestAdmin {
TEST_UTIL.getHBaseAdmin().createTable(htd);
}
@Test
@Test (timeout=300000)
public void testIsEnabledOrDisabledOnUnknownTable() throws Exception {
try {
admin.isTableEnabled(Bytes.toBytes("unkownTable"));

View File

@ -34,6 +34,8 @@ import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.log4j.Level;
import org.junit.AfterClass;
@ -89,7 +91,7 @@ public class TestClientScannerRPCTimeout {
putToTable(ht, r2);
putToTable(ht, r3);
LOG.info("Wrote our three values");
RegionServerWithScanTimeout.seqNoToSleepOn = 1;
RSRpcServicesWithScanTimeout.seqNoToSleepOn = 1;
Scan scan = new Scan();
scan.setCaching(1);
ResultScanner scanner = ht.getScanner(scan);
@ -100,15 +102,15 @@ public class TestClientScannerRPCTimeout {
result = scanner.next();
assertTrue((System.currentTimeMillis() - t1) > rpcTimeout);
assertTrue("Expected row: row-2", Bytes.equals(r2, result.getRow()));
RegionServerWithScanTimeout.seqNoToSleepOn = -1;// No need of sleep
RSRpcServicesWithScanTimeout.seqNoToSleepOn = -1;// No need of sleep
result = scanner.next();
assertTrue("Expected row: row-3", Bytes.equals(r3, result.getRow()));
scanner.close();
// test the case that RPC is always timesout
scanner = ht.getScanner(scan);
RegionServerWithScanTimeout.sleepAlways = true;
RegionServerWithScanTimeout.tryNumber = 0;
RSRpcServicesWithScanTimeout.sleepAlways = true;
RSRpcServicesWithScanTimeout.tryNumber = 0;
try {
result = scanner.next();
} catch (IOException ioe) {
@ -116,8 +118,8 @@ public class TestClientScannerRPCTimeout {
LOG.info("Failed after maximal attempts=" + CLIENT_RETRIES_NUMBER, ioe);
}
assertTrue("Expected maximal try number=" + CLIENT_RETRIES_NUMBER
+ ", actual =" + RegionServerWithScanTimeout.tryNumber,
RegionServerWithScanTimeout.tryNumber <= CLIENT_RETRIES_NUMBER);
+ ", actual =" + RSRpcServicesWithScanTimeout.tryNumber,
RSRpcServicesWithScanTimeout.tryNumber <= CLIENT_RETRIES_NUMBER);
}
private void putToTable(HTable ht, byte[] rowkey) throws IOException {
@ -127,15 +129,26 @@ public class TestClientScannerRPCTimeout {
}
private static class RegionServerWithScanTimeout extends MiniHBaseClusterRegionServer {
public RegionServerWithScanTimeout(Configuration conf)
throws IOException, InterruptedException {
super(conf);
}
protected RSRpcServices createRpcServices() throws IOException {
return new RSRpcServicesWithScanTimeout(this);
}
}
private static class RSRpcServicesWithScanTimeout extends RSRpcServices {
private long tableScannerId;
private boolean slept;
private static long seqNoToSleepOn = -1;
private static boolean sleepAlways = false;
private static int tryNumber = 0;
public RegionServerWithScanTimeout(Configuration conf)
throws IOException, InterruptedException {
super(conf);
public RSRpcServicesWithScanTimeout(HRegionServer rs)
throws IOException {
super(rs);
}
@Override

View File

@ -4158,7 +4158,7 @@ public class TestFromClientSide {
// HBaseAdmin and can connect to the new master;
HBaseAdmin newAdmin = new HBaseAdmin(conn);
assertTrue(newAdmin.tableExists(tableName));
assert(newAdmin.getClusterStatus().getServersSize() == SLAVES);
assertTrue(newAdmin.getClusterStatus().getServersSize() == SLAVES + 1);
}
@Test

View File

@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.exceptions.OperationConflictException;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
@ -277,7 +276,7 @@ public class TestMultiParallel {
// after writing successfully. It means the server we aborted is dead
// and detected by matser
while (liveRS.getRegionServer().getNumberOfOnlineRegions() != 0) {
Thread.sleep(10);
Thread.sleep(100);
}
// try putting more keys after the abort. same key/qual... just validating
// no exceptions thrown
@ -302,17 +301,14 @@ public class TestMultiParallel {
LOG.info("Count=" + count);
Assert.assertEquals("Server count=" + count + ", abort=" + doAbort,
(doAbort ? (liveRScount - 1) : liveRScount), count);
for (JVMClusterUtil.RegionServerThread t: liveRSs) {
int regions = ProtobufUtil.getOnlineRegions(t.getRegionServer()).size();
// Assert.assertTrue("Count of regions=" + regions, regions > 10);
}
if (doAbort) {
UTIL.getMiniHBaseCluster().waitOnRegionServer(0);
UTIL.waitFor(15 * 1000, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
// Master is also a regionserver, so the count is liveRScount
return UTIL.getMiniHBaseCluster().getMaster()
.getClusterStatus().getServersSize() == (liveRScount - 1);
.getClusterStatus().getServersSize() == liveRScount;
}
});
UTIL.waitFor(15 * 1000, UTIL.predicateNoRegionsInTransition());

View File

@ -472,7 +472,8 @@ public class TestScannersFromClientSide {
byte[] regionName = hri.getRegionName();
int i = cluster.getServerWith(regionName);
HRegionServer rs = cluster.getRegionServer(i);
ProtobufUtil.closeRegion(rs, rs.getServerName(), regionName, false);
ProtobufUtil.closeRegion(
rs.getRSRpcServices(), rs.getServerName(), regionName, false);
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long timeOut = 300000;
while (true) {
@ -492,7 +493,7 @@ public class TestScannersFromClientSide {
states.regionOffline(hri);
states.updateRegionState(hri, State.OPENING);
ZKAssign.createNodeOffline(zkw, hri, loc.getServerName());
ProtobufUtil.openRegion(rs, rs.getServerName(), hri);
ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri);
startTime = EnvironmentEdgeManager.currentTimeMillis();
while (true) {
if (rs.getOnlineRegion(regionName) != null) {

View File

@ -525,7 +525,7 @@ public class TestClassLoading {
"[" + masterCoprocessor.getSimpleName() + "]";
String loadedMasterCoprocessors =
java.util.Arrays.toString(
TEST_UTIL.getHBaseCluster().getMaster().getCoprocessors());
TEST_UTIL.getHBaseCluster().getMaster().getMasterCoprocessors());
assertEquals(loadedMasterCoprocessorsVerify, loadedMasterCoprocessors);
}

View File

@ -129,7 +129,7 @@ public class TestHTableWrapper {
@Test
public void testHTableInterfaceMethods() throws Exception {
Configuration conf = util.getConfiguration();
MasterCoprocessorHost cpHost = util.getMiniHBaseCluster().getMaster().getCoprocessorHost();
MasterCoprocessorHost cpHost = util.getMiniHBaseCluster().getMaster().getMasterCoprocessorHost();
Class<?> implClazz = DummyRegionObserver.class;
cpHost.load(implClazz, Coprocessor.PRIORITY_HIGHEST, conf);
CoprocessorEnvironment env = cpHost.findCoprocessorEnvironment(implClazz.getName());

View File

@ -160,7 +160,7 @@ public class TestMasterCoprocessorExceptionWithAbort {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
MasterCoprocessorHost host = master.getCoprocessorHost();
MasterCoprocessorHost host = master.getMasterCoprocessorHost();
BuggyMasterObserver cp = (BuggyMasterObserver)host.findCoprocessor(
BuggyMasterObserver.class.getName());
assertFalse("No table created yet", cp.wasCreateTableCalled());

View File

@ -141,7 +141,7 @@ public class TestMasterCoprocessorExceptionWithRemove {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
MasterCoprocessorHost host = master.getCoprocessorHost();
MasterCoprocessorHost host = master.getMasterCoprocessorHost();
BuggyMasterObserver cp = (BuggyMasterObserver)host.findCoprocessor(
BuggyMasterObserver.class.getName());
assertFalse("No table created yet", cp.wasCreateTableCalled());

View File

@ -34,7 +34,16 @@ import java.util.concurrent.CountDownLatch;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.master.AssignmentManager;
@ -42,10 +51,10 @@ import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
@ -968,7 +977,7 @@ public class TestMasterObserver {
HMaster master = cluster.getMaster();
assertTrue("Master should be active", master.isActiveMaster());
MasterCoprocessorHost host = master.getCoprocessorHost();
MasterCoprocessorHost host = master.getMasterCoprocessorHost();
assertNotNull("CoprocessorHost should not be null", host);
CPMasterObserver cp = (CPMasterObserver)host.findCoprocessor(
CPMasterObserver.class.getName());
@ -987,7 +996,7 @@ public class TestMasterObserver {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
MasterCoprocessorHost host = master.getCoprocessorHost();
MasterCoprocessorHost host = master.getMasterCoprocessorHost();
CPMasterObserver cp = (CPMasterObserver)host.findCoprocessor(
CPMasterObserver.class.getName());
cp.enableBypass(true);
@ -1147,7 +1156,7 @@ public class TestMasterObserver {
public void testSnapshotOperations() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
MasterCoprocessorHost host = master.getCoprocessorHost();
MasterCoprocessorHost host = master.getMasterCoprocessorHost();
CPMasterObserver cp = (CPMasterObserver)host.findCoprocessor(
CPMasterObserver.class.getName());
cp.resetStates();
@ -1204,7 +1213,7 @@ public class TestMasterObserver {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
String testNamespace = "observed_ns";
HMaster master = cluster.getMaster();
MasterCoprocessorHost host = master.getCoprocessorHost();
MasterCoprocessorHost host = master.getMasterCoprocessorHost();
CPMasterObserver cp = (CPMasterObserver)host.findCoprocessor(
CPMasterObserver.class.getName());
@ -1269,7 +1278,7 @@ public class TestMasterObserver {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
MasterCoprocessorHost host = master.getCoprocessorHost();
MasterCoprocessorHost host = master.getMasterCoprocessorHost();
CPMasterObserver cp = (CPMasterObserver)host.findCoprocessor(
CPMasterObserver.class.getName());
cp.enableBypass(false);
@ -1308,8 +1317,8 @@ public class TestMasterObserver {
}
assertTrue("Found server", found);
LOG.info("Found " + destName);
master.moveRegion(null,RequestConverter.buildMoveRegionRequest(
firstGoodPair.getKey().getEncodedNameAsBytes(),Bytes.toBytes(destName)));
master.getMasterRpcServices().moveRegion(null, RequestConverter.buildMoveRegionRequest(
firstGoodPair.getKey().getEncodedNameAsBytes(),Bytes.toBytes(destName)));
assertTrue("Coprocessor should have been called on region move",
cp.wasMoveCalled());
@ -1334,13 +1343,13 @@ public class TestMasterObserver {
byte[] destRS = Bytes.toBytes(cluster.getRegionServer(1).getServerName().toString());
//Make sure no regions are in transition now
waitForRITtoBeZero(master);
List<HRegionInfo> openRegions = ProtobufUtil.getOnlineRegions(rs);
List<HRegionInfo> openRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
int moveCnt = openRegions.size()/2;
for (int i=0; i<moveCnt; i++) {
HRegionInfo info = openRegions.get(i);
if (!info.isMetaTable()) {
master.moveRegion(null,RequestConverter.buildMoveRegionRequest(
openRegions.get(i).getEncodedNameAsBytes(), destRS));
master.getMasterRpcServices().moveRegion(null, RequestConverter.buildMoveRegionRequest(
openRegions.get(i).getEncodedNameAsBytes(), destRS));
}
}
//Make sure no regions are in transition now
@ -1370,14 +1379,14 @@ public class TestMasterObserver {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
MasterCoprocessorHost host = master.getCoprocessorHost();
MasterCoprocessorHost host = master.getMasterCoprocessorHost();
CPMasterObserver cp = (CPMasterObserver)host.findCoprocessor(
CPMasterObserver.class.getName());
cp.resetStates();
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest((List<TableName>)null);
master.getTableDescriptors(null, req);
master.getMasterRpcServices().getTableDescriptors(null, req);
assertTrue("Coprocessor should be called on table descriptors request",
cp.wasGetTableDescriptorsCalled());

View File

@ -586,7 +586,7 @@ public class TestRegionObserverInterface {
if (!t.isAlive() || t.getRegionServer().isAborted() || t.getRegionServer().isStopping()){
continue;
}
for (HRegionInfo r : ProtobufUtil.getOnlineRegions(t.getRegionServer())) {
for (HRegionInfo r : ProtobufUtil.getOnlineRegions(t.getRegionServer().getRSRpcServices())) {
if (!r.getTable().equals(tableName)) {
continue;
}

View File

@ -64,7 +64,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
final HRegionServer regionServer = cluster.getRegionServer(0);
conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
FailedInitializationObserver.class.getName());
regionServer.getCoprocessorHost().loadSystemCoprocessors(conf,
regionServer.getRegionServerCoprocessorHost().loadSystemCoprocessors(conf,
CoprocessorHost.REGION_COPROCESSOR_CONF_KEY);
TEST_UTIL.waitFor(10000, 1000, new Predicate<Exception>() {
@Override
@ -98,20 +98,16 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
// Note which regionServer will abort (after put is attempted).
final HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
boolean threwIOE = false;
try {
final byte[] ROW = Bytes.toBytes("aaa");
Put put = new Put(ROW);
put.add(TEST_FAMILY, ROW, ROW);
table.put(put);
table.flushCommits();
// We may need two puts to reliably get an exception
table.put(put);
table.flushCommits();
} catch (IOException e) {
threwIOE = true;
} finally {
assertTrue("The regionserver should have thrown an exception", threwIOE);
// The region server is going to be aborted.
// We may get an exception if we retry,
// which is not guaranteed.
}
// Wait 10 seconds for the regionserver to abort: expected result is that

View File

@ -31,17 +31,13 @@ import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction;
@ -82,7 +78,7 @@ public class TestRegionServerObserver {
try {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HRegionServer regionServer = cluster.getRegionServer(0);
RegionServerCoprocessorHost cpHost = regionServer.getCoprocessorHost();
RegionServerCoprocessorHost cpHost = regionServer.getRegionServerCoprocessorHost();
Coprocessor coprocessor = cpHost.findCoprocessor(CPRegionServerObserver.class.getName());
CPRegionServerObserver regionServerObserver = (CPRegionServerObserver) coprocessor;
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(TABLENAME));
@ -115,7 +111,6 @@ public class TestRegionServerObserver {
private RegionMergeTransaction rmt = null;
private HRegion mergedRegion = null;
private boolean bypass = false;
private boolean preMergeCalled;
private boolean preMergeBeforePONRCalled;
private boolean preMergeAfterPONRCalled;
@ -123,10 +118,6 @@ public class TestRegionServerObserver {
private boolean postRollBackMergeCalled;
private boolean postMergeCalled;
public void enableBypass(boolean bypass) {
this.bypass = bypass;
}
public void resetStates() {
preMergeCalled = false;
preMergeBeforePONRCalled = false;

View File

@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
@ -244,19 +243,18 @@ public class TestBlockReorder {
byte[] sb = "sb".getBytes();
htu.startMiniZKCluster();
MiniHBaseCluster hbm = htu.startMiniHBaseCluster(1, 1);
MiniHBaseCluster hbm = htu.startMiniHBaseCluster(1, 0);
hbm.waitForActiveAndReadyMaster();
hbm.getRegionServer(0).waitForServerOnline();
HRegionServer targetRs = hbm.getMaster();
// We want to have a datanode with the same name as the region server, so
// we're going to get the regionservername, and start a new datanode with this name.
String host4 = hbm.getRegionServer(0).getServerName().getHostname();
String host4 = targetRs.getServerName().getHostname();
LOG.info("Starting a new datanode with the name=" + host4);
cluster.startDataNodes(conf, 1, true, null, new String[]{"/r4"}, new String[]{host4}, null);
cluster.waitClusterUp();
final int repCount = 3;
HRegionServer targetRs = hbm.getRegionServer(0);
// We use the regionserver file system & conf as we expect it to have the hook.
conf = targetRs.getConfiguration();

View File

@ -183,10 +183,11 @@ public class TestLoadIncrementalHFilesSplitRecovery {
HRegionServer hrs = util.getRSForFirstRegionInTable(Bytes
.toBytes(table));
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs)) {
for (HRegionInfo hri :
ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) {
// splitRegion doesn't work if startkey/endkey are null
ProtobufUtil.split(hrs, hri, rowkey(ROWCOUNT / 2)); // hard code split
ProtobufUtil.split(hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2)); // hard code split
}
}
@ -194,7 +195,8 @@ public class TestLoadIncrementalHFilesSplitRecovery {
int regions;
do {
regions = 0;
for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs)) {
for (HRegionInfo hri :
ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) {
regions++;
}
@ -298,6 +300,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
fail("doBulkLoad should have thrown an exception");
}
@SuppressWarnings("deprecation")
private HConnection getMockedConnection(final Configuration conf)
throws IOException, ServiceException {
HConnection c = Mockito.mock(HConnection.class);

View File

@ -18,23 +18,22 @@
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.net.InetSocketAddress;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentSkipListMap;
import com.google.protobuf.Message;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.Get;
@ -84,7 +83,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
import org.apache.hadoop.hbase.regionserver.FlushRequester;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -545,11 +543,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
return null;
}
@Override
public int getPriority(RPCProtos.RequestHeader header, Message param) {
return 0;
}
@Override
public UpdateFavoredNodesResponse updateFavoredNodes(RpcController controller,
UpdateFavoredNodesRequest request) throws ServiceException {

View File

@ -72,36 +72,6 @@ public class Mocking {
}
}
/**
* Fakes the regionserver-side zk transitions of a region open.
* @param w ZooKeeperWatcher to use.
* @param sn Name of the regionserver doing the 'opening'
* @param hri Region we're 'opening'.
* @throws KeeperException
* @throws DeserializationException
*/
static void fakeRegionServerRegionOpenInZK(HMaster master, final ZooKeeperWatcher w,
final ServerName sn, final HRegionInfo hri)
throws KeeperException, DeserializationException, InterruptedException {
// Wait till the we region is ready to be open in RIT.
waitForRegionPendingOpenInRIT(master.getAssignmentManager(), hri.getEncodedName());
// Get current versionid else will fail on transition from OFFLINE to OPENING below
int versionid = ZKAssign.getVersion(w, hri);
assertNotSame(-1, versionid);
// This uglyness below is what the openregionhandler on RS side does. I
// looked at exposing the method over in openregionhandler but its just a
// one liner and its deep over in another package so just repeat it below.
versionid = ZKAssign.transitionNode(w, hri, sn,
EventType.M_ZK_REGION_OFFLINE, EventType.RS_ZK_REGION_OPENING, versionid);
assertNotSame(-1, versionid);
// Move znode from OPENING to OPENED as RS does on successful open.
versionid = ZKAssign.transitionNodeOpened(w, hri, sn, versionid);
assertNotSame(-1, versionid);
// We should be done now. The master open handler will notice the
// transition and remove this regions znode.
}
/**
* Verifies that the specified region is in the specified state in ZooKeeper.
* <p>

View File

@ -86,7 +86,7 @@ public class TestActiveMasterManager {
MonitoredTask status = Mockito.mock(MonitoredTask.class);
clusterStatusTracker.setClusterUp();
activeMasterManager.blockUntilBecomingActiveMaster(status);
activeMasterManager.blockUntilBecomingActiveMaster(100, status);
assertTrue(activeMasterManager.clusterHasActiveMaster.get());
assertMaster(zk, master);
@ -95,7 +95,7 @@ public class TestActiveMasterManager {
ActiveMasterManager secondActiveMasterManager =
secondDummyMaster.getActiveMasterManager();
assertFalse(secondActiveMasterManager.clusterHasActiveMaster.get());
activeMasterManager.blockUntilBecomingActiveMaster(status);
activeMasterManager.blockUntilBecomingActiveMaster(100, status);
assertTrue(activeMasterManager.clusterHasActiveMaster.get());
assertMaster(zk, master);
}
@ -130,7 +130,7 @@ public class TestActiveMasterManager {
ClusterStatusTracker clusterStatusTracker =
ms1.getClusterStatusTracker();
clusterStatusTracker.setClusterUp();
activeMasterManager.blockUntilBecomingActiveMaster(
activeMasterManager.blockUntilBecomingActiveMaster(100,
Mockito.mock(MonitoredTask.class));
assertTrue(activeMasterManager.clusterHasActiveMaster.get());
assertMaster(zk, firstMasterAddress);
@ -212,7 +212,7 @@ public class TestActiveMasterManager {
@Override
public void run() {
manager.blockUntilBecomingActiveMaster(
manager.blockUntilBecomingActiveMaster(100,
Mockito.mock(MonitoredTask.class));
LOG.info("Second master has become the active master!");
isActiveMaster = true;
@ -308,6 +308,5 @@ public class TestActiveMasterManager {
return activeMasterManager;
}
}
}

View File

@ -197,7 +197,7 @@ public class TestCatalogJanitor {
private final AssignmentManager asm;
MockMasterServices(final Server server) throws IOException {
this.mfs = new MasterFileSystem(server, this, false);
this.mfs = new MasterFileSystem(server, this);
this.asm = Mockito.mock(AssignmentManager.class);
}
@ -228,7 +228,7 @@ public class TestCatalogJanitor {
}
@Override
public MasterCoprocessorHost getCoprocessorHost() {
public MasterCoprocessorHost getMasterCoprocessorHost() {
return null;
}

View File

@ -25,7 +25,6 @@ import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_acquired;
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_done;
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_err;
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_resigned;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@ -40,7 +39,6 @@ import java.util.LinkedList;
import java.util.List;
import java.util.NavigableSet;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
@ -55,10 +53,9 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
@ -68,10 +65,11 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.NonceGenerator;
@ -113,6 +111,7 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
@SuppressWarnings("deprecation")
public class TestDistributedLogSplitting {
private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
static {
@ -148,6 +147,7 @@ public class TestDistributedLogSplitting {
public static void tearDown() throws IOException {
TEST_UTIL.shutdownMiniZKCluster();
TEST_UTIL.shutdownMiniDFSCluster();
TEST_UTIL.shutdownMiniHBaseCluster();
}
private void startCluster(int num_rs) throws Exception {
@ -159,6 +159,7 @@ public class TestDistributedLogSplitting {
conf.setInt(HConstants.REGIONSERVER_INFO_PORT, -1);
conf.setFloat(HConstants.LOAD_BALANCER_SLOP_KEY, (float) 100.0); // no load balancing
conf.setInt("hbase.regionserver.wal.max.splitters", 3);
TEST_UTIL.shutdownMiniHBaseCluster();
TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.setDFSCluster(dfsCluster);
TEST_UTIL.setZkCluster(zkCluster);
@ -219,7 +220,7 @@ public class TestDistributedLogSplitting {
for (int i = 0; i < NUM_RS; i++) {
boolean foundRs = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
for (HRegionInfo region : regions) {
if (region.getTable().getNameAsString().equalsIgnoreCase("table")) {
foundRs = true;
@ -248,7 +249,6 @@ public class TestDistributedLogSplitting {
for (HRegionInfo hri : regions) {
Path tdir = FSUtils.getTableDir(rootdir, table);
@SuppressWarnings("deprecation")
Path editsdir =
HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName()));
LOG.debug("checking edits dir " + editsdir);
@ -279,7 +279,7 @@ public class TestDistributedLogSplitting {
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
HRegionServer hrs = findRSToKill(false, "table");
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
// wait for abort completes
@ -328,7 +328,7 @@ public class TestDistributedLogSplitting {
List<Increment> reqs = new ArrayList<Increment>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
HRegionServer hrs = rst.getRegionServer();
List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs);
List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
for (HRegionInfo hri : hris) {
if (TABLE_NAME.equalsIgnoreCase(hri.getTable().getNameAsString())) {
byte[] key = hri.getStartKey();
@ -377,7 +377,7 @@ public class TestDistributedLogSplitting {
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
HRegionServer hrs = findRSToKill(true, "table");
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES);
@ -412,7 +412,8 @@ public class TestDistributedLogSplitting {
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (getAllOnlineRegions(tmpCluster).size() >= (numRegions + 1));
return (HBaseTestingUtility.getAllOnlineRegions(tmpCluster).size()
>= (numRegions + 1));
}
});
@ -440,12 +441,11 @@ public class TestDistributedLogSplitting {
// they will consume recovered.edits
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
HRegionServer hrs = findRSToKill(false, "table");
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
// abort master
@ -464,20 +464,20 @@ public class TestDistributedLogSplitting {
});
Thread.sleep(2000);
LOG.info("Current Open Regions:" + getAllOnlineRegions(cluster).size());
startMasterAndWaitUntilLogSplit(cluster);
LOG.info("Current Open Regions:"
+ HBaseTestingUtility.getAllOnlineRegions(cluster).size());
// wait for abort completes
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1));
return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
>= (NUM_REGIONS_TO_CREATE + 1));
}
});
LOG.info("Current Open Regions After Master Node Starts Up:"
+ getAllOnlineRegions(cluster).size());
+ HBaseTestingUtility.getAllOnlineRegions(cluster).size());
assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
@ -498,12 +498,11 @@ public class TestDistributedLogSplitting {
// they will consume recovered.edits
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
HRegionServer hrs = findRSToKill(false, "table");
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
// abort master
@ -522,9 +521,8 @@ public class TestDistributedLogSplitting {
});
Thread.sleep(2000);
LOG.info("Current Open Regions:" + getAllOnlineRegions(cluster).size());
startMasterAndWaitUntilLogSplit(cluster);
LOG.info("Current Open Regions:"
+ HBaseTestingUtility.getAllOnlineRegions(cluster).size());
// wait for all regions are fully recovered
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@ -537,7 +535,7 @@ public class TestDistributedLogSplitting {
});
LOG.info("Current Open Regions After Master Node Starts Up:"
+ getAllOnlineRegions(cluster).size());
+ HBaseTestingUtility.getAllOnlineRegions(cluster).size());
assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
@ -563,7 +561,7 @@ public class TestDistributedLogSplitting {
List<HRegionInfo> regions = null;
HRegionServer hrs1 = findRSToKill(false, "table");
regions = ProtobufUtil.getOnlineRegions(hrs1);
regions = ProtobufUtil.getOnlineRegions(hrs1.getRSRpcServices());
makeHLog(hrs1.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
@ -583,7 +581,8 @@ public class TestDistributedLogSplitting {
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1));
return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
>= (NUM_REGIONS_TO_CREATE + 1));
}
});
@ -607,7 +606,8 @@ public class TestDistributedLogSplitting {
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1));
return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
>= (NUM_REGIONS_TO_CREATE + 1));
}
});
@ -633,7 +633,7 @@ public class TestDistributedLogSplitting {
startCluster(NUM_RS);
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = master.getZooKeeperWatcher();
final ZooKeeperWatcher zkw = master.getZooKeeper();
HTable ht = installTable(zkw, "table", "family", 40);
final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
@ -644,7 +644,7 @@ public class TestDistributedLogSplitting {
ServerName secondFailedServer = null;
for (int i = 0; i < NUM_RS; i++) {
hrs = rsts.get(i).getRegionServer();
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
if (regions.isEmpty()) continue;
region = regions.get(0);
regionSet.add(region);
@ -670,7 +670,6 @@ public class TestDistributedLogSplitting {
}
});
ht.close();
zkw.close();
}
@Test(timeout = 300000)
@ -692,7 +691,7 @@ public class TestDistributedLogSplitting {
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
for (HRegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
@ -744,7 +743,7 @@ public class TestDistributedLogSplitting {
hasRegionsForBothTables = false;
boolean isCarryingSystem = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
for (HRegionInfo region : regions) {
if (region.getTable().isSystemTable()) {
isCarryingSystem = true;
@ -800,7 +799,8 @@ public class TestDistributedLogSplitting {
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1));
return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
>= (NUM_REGIONS_TO_CREATE + 1));
}
});
@ -821,7 +821,6 @@ public class TestDistributedLogSplitting {
Path rootdir = FSUtils.getRootDir(conf);
Path tdir = FSUtils.getTableDir(rootdir, TableName.valueOf("disableTable"));
for (HRegionInfo hri : regions) {
@SuppressWarnings("deprecation")
Path editsdir =
HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName()));
LOG.debug("checking edits dir " + editsdir);
@ -843,7 +842,6 @@ public class TestDistributedLogSplitting {
// clean up
for (HRegionInfo hri : regions) {
@SuppressWarnings("deprecation")
Path editsdir =
HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName()));
fs.delete(editsdir, true);
@ -876,7 +874,7 @@ public class TestDistributedLogSplitting {
HRegionServer dstRS = null;
for (int i = 0; i < NUM_RS; i++) {
hrs = rsts.get(i).getRegionServer();
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
if (regions.isEmpty()) continue;
region = regions.get(0);
regionSet.add(region);
@ -910,7 +908,6 @@ public class TestDistributedLogSplitting {
Put put = new Put(key);
put.add(Bytes.toBytes("family"), Bytes.toBytes("c1"), new byte[]{'b'});
ht.put(put);
ht.close();
} catch (IOException ioe) {
Assert.assertTrue(ioe instanceof RetriesExhaustedWithDetailsException);
RetriesExhaustedWithDetailsException re = (RetriesExhaustedWithDetailsException) ioe;
@ -926,6 +923,7 @@ public class TestDistributedLogSplitting {
foundRegionInRecoveryException);
}
ht.close();
zkw.close();
}
@ -955,8 +953,8 @@ public class TestDistributedLogSplitting {
installTable(new ZooKeeperWatcher(conf, "table-creation", null),
"table", "family", 40);
makeHLog(hrs.getWAL(), ProtobufUtil.getOnlineRegions(hrs), "table", "family", NUM_LOG_LINES,
100);
makeHLog(hrs.getWAL(), ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()),
"table", "family", NUM_LOG_LINES, 100);
new Thread() {
public void run() {
@ -1025,7 +1023,8 @@ public class TestDistributedLogSplitting {
}
start = EnvironmentEdgeManager.currentTimeMillis();
while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 1)) {
while (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
< (NUM_REGIONS_TO_CREATE + 1)) {
if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
assertTrue("Timedout", false);
}
@ -1124,11 +1123,10 @@ public class TestDistributedLogSplitting {
// they will consume recovered.edits
master.balanceSwitch(false);
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
// only testing meta recovery in ZK operation
HRegionServer hrs = findRSToKill(true, null);
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
LOG.info("#regions = " + regions.size());
Set<HRegionInfo> tmpRegions = new HashSet<HRegionInfo>();
@ -1187,7 +1185,7 @@ public class TestDistributedLogSplitting {
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
for (HRegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
@ -1279,7 +1277,7 @@ public class TestDistributedLogSplitting {
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
for (HRegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
@ -1377,7 +1375,7 @@ public class TestDistributedLogSplitting {
TEST_UTIL.getHBaseAdmin().disableTable(table);
LOG.debug("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
NavigableSet<String> regions = getAllOnlineRegions(cluster);
NavigableSet<String> regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
LOG.debug("Verifying only catalog and namespace regions are assigned\n");
if (regions.size() != 2) {
for (String oregion : regions)
@ -1389,7 +1387,7 @@ public class TestDistributedLogSplitting {
LOG.debug("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
regions = getAllOnlineRegions(cluster);
regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
assertEquals(numRegions + 2 + existingRegions, regions.size());
return ht;
}
@ -1402,7 +1400,7 @@ public class TestDistributedLogSplitting {
for (RegionServerThread rst : rsts) {
HRegionServer hrs = rst.getRegionServer();
List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs);
List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
for (HRegionInfo hri : hris) {
if (hri.getTable().isSystemTable()) {
continue;
@ -1414,6 +1412,21 @@ public class TestDistributedLogSplitting {
putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
}
}
for (MasterThread mt : cluster.getLiveMasterThreads()) {
HRegionServer hrs = mt.getMaster();
List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
for (HRegionInfo hri : hris) {
if (hri.getTable().isSystemTable()) {
continue;
}
LOG.debug("adding data to rs = " + mt.getName() +
" region = "+ hri.getRegionNameAsString());
HRegion region = hrs.getOnlineRegion(hri.getRegionName());
assertTrue(region != null);
putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
}
}
}
public void makeHLog(HLog log, List<HRegionInfo> regions, String tname, String fname,
@ -1542,17 +1555,6 @@ public class TestDistributedLogSplitting {
t.flushCommits();
}
private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
throws IOException {
NavigableSet<String> online = new TreeSet<String>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
for (HRegionInfo region : ProtobufUtil.getOnlineRegions(rst.getRegionServer())) {
online.add(region.getRegionNameAsString());
}
}
return online;
}
private void waitForCounter(AtomicLong ctr, long oldval, long newval,
long timems) {
long curt = System.currentTimeMillis();
@ -1580,19 +1582,6 @@ public class TestDistributedLogSplitting {
LOG.debug("Master is aborted");
}
private void startMasterAndWaitUntilLogSplit(MiniHBaseCluster cluster)
throws IOException, InterruptedException {
cluster.startMaster();
HMaster master = cluster.getMaster();
while (!master.isInitialized()) {
Thread.sleep(100);
}
ServerManager serverManager = master.getServerManager();
while (serverManager.areDeadServersInProgress()) {
Thread.sleep(100);
}
}
/**
* Find a RS that has regions of a table.
* @param hasMetaRegion when true, the returned RS has hbase:meta region as well
@ -1610,7 +1599,7 @@ public class TestDistributedLogSplitting {
boolean isCarryingMeta = false;
boolean foundTableRegion = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
for (HRegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;

View File

@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKTable;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -284,6 +285,20 @@ public class TestMasterFailover {
* javadoc.
*/
// Master is down, so is the meta. We need to assign it somewhere
// so that regions can be assigned during the mocking phase.
ZKAssign.createNodeOffline(
zkw, HRegionInfo.FIRST_META_REGIONINFO, hrs.getServerName());
ProtobufUtil.openRegion(hrs.getRSRpcServices(),
hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
while (true) {
ServerName sn = MetaRegionTracker.getMetaRegionLocation(zkw);
if (sn != null && sn.equals(hrs.getServerName())) {
break;
}
Thread.sleep(100);
}
List<HRegionInfo> regionsThatShouldBeOnline = new ArrayList<HRegionInfo>();
List<HRegionInfo> regionsThatShouldBeOffline = new ArrayList<HRegionInfo>();
@ -337,7 +352,7 @@ public class TestMasterFailover {
region = enabledRegions.remove(0);
regionsThatShouldBeOnline.add(region);
ZKAssign.createNodeOffline(zkw, region, serverName);
ProtobufUtil.openRegion(hrs, hrs.getServerName(), region);
ProtobufUtil.openRegion(hrs.getRSRpcServices(), hrs.getServerName(), region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -352,7 +367,7 @@ public class TestMasterFailover {
region = disabledRegions.remove(0);
regionsThatShouldBeOffline.add(region);
ZKAssign.createNodeOffline(zkw, region, serverName);
ProtobufUtil.openRegion(hrs, hrs.getServerName(), region);
ProtobufUtil.openRegion(hrs.getRSRpcServices(), hrs.getServerName(), region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -407,7 +422,8 @@ public class TestMasterFailover {
Set<HRegionInfo> onlineRegions = new TreeSet<HRegionInfo>();
for (JVMClusterUtil.RegionServerThread rst :
cluster.getRegionServerThreads()) {
onlineRegions.addAll(ProtobufUtil.getOnlineRegions(rst.getRegionServer()));
onlineRegions.addAll(ProtobufUtil.getOnlineRegions(
rst.getRegionServer().getRSRpcServices()));
}
// Now, everything that should be online should be online
@ -658,6 +674,20 @@ public class TestMasterFailover {
* javadoc.
*/
// Master is down, so is the meta. We need to assign it somewhere
// so that regions can be assigned during the mocking phase.
ZKAssign.createNodeOffline(
zkw, HRegionInfo.FIRST_META_REGIONINFO, hrs.getServerName());
ProtobufUtil.openRegion(hrs.getRSRpcServices(),
hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
while (true) {
ServerName sn = MetaRegionTracker.getMetaRegionLocation(zkw);
if (sn != null && sn.equals(hrs.getServerName())) {
break;
}
Thread.sleep(100);
}
List<HRegionInfo> regionsThatShouldBeOnline = new ArrayList<HRegionInfo>();
List<HRegionInfo> regionsThatShouldBeOffline = new ArrayList<HRegionInfo>();
@ -736,7 +766,8 @@ public class TestMasterFailover {
region = enabledRegions.remove(0);
regionsThatShouldBeOnline.add(region);
ZKAssign.createNodeOffline(zkw, region, deadServerName);
ProtobufUtil.openRegion(hrsDead, hrsDead.getServerName(), region);
ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
hrsDead.getServerName(), region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -752,7 +783,8 @@ public class TestMasterFailover {
region = disabledRegions.remove(0);
regionsThatShouldBeOffline.add(region);
ZKAssign.createNodeOffline(zkw, region, deadServerName);
ProtobufUtil.openRegion(hrsDead, hrsDead.getServerName(), region);
ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
hrsDead.getServerName(), region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -772,7 +804,8 @@ public class TestMasterFailover {
region = enabledRegions.remove(0);
regionsThatShouldBeOnline.add(region);
ZKAssign.createNodeOffline(zkw, region, deadServerName);
ProtobufUtil.openRegion(hrsDead, hrsDead.getServerName(), region);
ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
hrsDead.getServerName(), region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -790,7 +823,8 @@ public class TestMasterFailover {
region = disabledRegions.remove(0);
regionsThatShouldBeOffline.add(region);
ZKAssign.createNodeOffline(zkw, region, deadServerName);
ProtobufUtil.openRegion(hrsDead, hrsDead.getServerName(), region);
ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
hrsDead.getServerName(), region);
while (true) {
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
RegionTransition rt = RegionTransition.parseFrom(bytes);
@ -863,7 +897,7 @@ public class TestMasterFailover {
}
Thread.sleep(100);
}
onlineRegions.addAll(ProtobufUtil.getOnlineRegions(rs));
onlineRegions.addAll(ProtobufUtil.getOnlineRegions(rs.getRSRpcServices()));
} catch (RegionServerStoppedException e) {
LOG.info("Got RegionServerStoppedException", e);
}
@ -891,7 +925,8 @@ public class TestMasterFailover {
*/
private void verifyRegionLocation(HRegionServer hrs, List<HRegionInfo> regions)
throws IOException {
List<HRegionInfo> tmpOnlineRegions = ProtobufUtil.getOnlineRegions(hrs);
List<HRegionInfo> tmpOnlineRegions =
ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
Iterator<HRegionInfo> itr = regions.iterator();
while (itr.hasNext()) {
HRegionInfo tmp = itr.next();
@ -937,18 +972,9 @@ public class TestMasterFailover {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
// Find regionserver carrying meta.
List<RegionServerThread> regionServerThreads =
cluster.getRegionServerThreads();
int count = -1;
HRegion metaRegion = null;
for (RegionServerThread regionServerThread : regionServerThreads) {
HRegionServer regionServer = regionServerThread.getRegionServer();
metaRegion = regionServer.getOnlineRegion(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
count++;
regionServer.abort("");
if (null != metaRegion) break;
}
HRegionServer regionServer = cluster.getRegionServer(count);
HRegionServer regionServer = cluster.getMaster();
HRegion metaRegion = regionServer.getOnlineRegion(
HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
TEST_UTIL.shutdownMiniHBaseCluster();
@ -1114,7 +1140,7 @@ public class TestMasterFailover {
assertEquals(2, masterThreads.size());
int rsCount = masterThreads.get(activeIndex).getMaster().getClusterStatus().getServersSize();
LOG.info("Active master " + active.getServerName() + " managing " + rsCount + " regions servers");
assertEquals(3, rsCount);
assertEquals(5, rsCount);
// Check that ClusterStatus reports the correct active and backup masters
assertNotNull(active);
@ -1147,7 +1173,7 @@ public class TestMasterFailover {
int rss = status.getServersSize();
LOG.info("Active master " + mastername.getServerName() + " managing " +
rss + " region servers");
assertEquals(3, rss);
assertEquals(4, rss);
// Stop the cluster
TEST_UTIL.shutdownMiniCluster();

View File

@ -27,11 +27,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -48,11 +48,11 @@ public class TestMasterMetrics {
private static HMaster master;
private static HBaseTestingUtility TEST_UTIL;
private static class MyRegionServer extends MiniHBaseClusterRegionServer {
public MyRegionServer(Configuration conf) throws IOException,
InterruptedException {
public static class MyMaster extends HMaster {
public MyMaster(Configuration conf) throws IOException,
KeeperException, InterruptedException {
super(conf);
}
}
@Override
protected void tryRegionServerReport(
@ -65,7 +65,7 @@ public class TestMasterMetrics {
public static void startCluster() throws Exception {
LOG.info("Starting cluster");
TEST_UTIL = new HBaseTestingUtility();
TEST_UTIL.startMiniCluster(1, 1, 1, null, HMaster.class, MyRegionServer.class);
TEST_UTIL.startMiniCluster(1, 0, 1, null, MyMaster.class, null);
cluster = TEST_UTIL.getHBaseCluster();
LOG.info("Waiting for active/ready master");
cluster.waitForActiveAndReadyMaster();
@ -85,16 +85,16 @@ public class TestMasterMetrics {
// sending fake request to master to see how metric value has changed
RegionServerStatusProtos.RegionServerReportRequest.Builder request =
RegionServerStatusProtos.RegionServerReportRequest.newBuilder();
ServerName serverName = cluster.getRegionServer(0).getServerName();
ServerName serverName = cluster.getMaster(0).getServerName();
request.setServer(ProtobufUtil.toServerName(serverName));
MetricsMasterSource masterSource = master.getMetrics().getMetricsSource();
MetricsMasterSource masterSource = master.getMasterMetrics().getMetricsSource();
ClusterStatusProtos.ServerLoad sl = ClusterStatusProtos.ServerLoad.newBuilder()
.setTotalNumberOfRequests(10000)
.build();
masterSource.init();
request.setLoad(sl);
master.regionServerReport(null, request.build());
master.getMasterRpcServices().regionServerReport(null, request.build());
metricsHelper.assertCounter("cluster_requests", 10000, masterSource);
@ -102,11 +102,11 @@ public class TestMasterMetrics {
.setTotalNumberOfRequests(15000)
.build();
request.setLoad(sl);
master.regionServerReport(null, request.build());
master.getMasterRpcServices().regionServerReport(null, request.build());
metricsHelper.assertCounter("cluster_requests", 15000, masterSource);
master.regionServerReport(null, request.build());
master.getMasterRpcServices().regionServerReport(null, request.build());
metricsHelper.assertCounter("cluster_requests", 15000, masterSource);
master.stopMaster();
@ -114,7 +114,7 @@ public class TestMasterMetrics {
@Test
public void testDefaultMasterMetrics() throws Exception {
MetricsMasterSource masterSource = master.getMetrics().getMetricsSource();
MetricsMasterSource masterSource = master.getMasterMetrics().getMetricsSource();
metricsHelper.assertGauge( "numRegionServers", 1, masterSource);
metricsHelper.assertGauge( "averageLoad", 2, masterSource);
metricsHelper.assertGauge( "numDeadRegionServers", 0, masterSource);

View File

@ -55,9 +55,9 @@ public class TestMasterMetricsWrapper {
assertEquals(master.getClusterId(), info.getClusterId());
assertEquals(master.getMasterActiveTime(), info.getActiveTime());
assertEquals(master.getMasterStartTime(), info.getStartTime());
assertEquals(master.getCoprocessors().length, info.getCoprocessors().length);
assertEquals(master.getMasterCoprocessors().length, info.getCoprocessors().length);
assertEquals(master.getServerManager().getOnlineServersList().size(), info.getNumRegionServers());
assertTrue(info.getNumRegionServers() == 4);
assertEquals(5, info.getNumRegionServers());
String zkServers = info.getZookeeperQuorum();
assertEquals(zkServers.split(",").length, TEST_UTIL.getZkCluster().getZooKeeperServerNum());
@ -69,10 +69,10 @@ public class TestMasterMetricsWrapper {
// We stopped the regionserver but could take a while for the master to notice it so hang here
// until it does... then move forward to see if metrics wrapper notices.
while (TEST_UTIL.getHBaseCluster().getMaster().getServerManager().getOnlineServers().size() !=
index) {
4) {
Threads.sleep(10);
}
assertTrue(info.getNumRegionServers() == 3);
assertTrue(info.getNumDeadRegionServers() == 1);
assertEquals(4, info.getNumRegionServers());
assertEquals(1, info.getNumDeadRegionServers());
}
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@ -29,16 +28,18 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaMockingUtil;
@ -46,28 +47,24 @@ import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
import com.google.protobuf.ServiceException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.Mockito;
import org.junit.experimental.categories.Category;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.mockito.Mockito;
import com.google.protobuf.ServiceException;
/**
* Standup the master and fake it to test various aspects of master function.
@ -87,6 +84,7 @@ public class TestMasterNoCluster {
Configuration c = TESTUTIL.getConfiguration();
// We use local filesystem. Set it so it writes into the testdir.
FSUtils.setRootDir(c, TESTUTIL.getDataTestDir());
DefaultMetricsSystem.setMiniClusterMode(true);
// Startup a mini zk cluster.
TESTUTIL.startMiniZKCluster();
}
@ -179,6 +177,9 @@ public class TestMasterNoCluster {
InetAddress getRemoteInetAddress(final int port, final long serverStartCode)
throws UnknownHostException {
// Return different address dependent on port passed.
if (port > sns.length) {
return super.getRemoteInetAddress(port, serverStartCode);
}
ServerName sn = sns[port];
return InetAddress.getByAddress(sn.getHostname(),
new byte [] {10, 0, 0, (byte)sn.getPort()});
@ -190,17 +191,15 @@ public class TestMasterNoCluster {
ServerManager sm = super.createServerManager(master, services);
// Spy on the created servermanager
ServerManager spy = Mockito.spy(sm);
// Fake a successful open.
Mockito.doReturn(RegionOpeningState.OPENED).when(spy).
sendRegionOpen((ServerName)Mockito.any(), (HRegionInfo)Mockito.any(),
Mockito.anyInt(), Mockito.anyListOf(ServerName.class));
// Fake a successful close.
Mockito.doReturn(true).when(spy).
sendRegionClose((ServerName)Mockito.any(), (HRegionInfo)Mockito.any(),
Mockito.anyInt(), (ServerName)Mockito.any(), Mockito.anyBoolean());
return spy;
}
@Override
CatalogTracker createCatalogTracker(ZooKeeperWatcher zk,
Configuration conf, Abortable abortable)
throws IOException {
protected CatalogTracker createCatalogTracker() throws IOException {
// Insert a mock for the connection used by the CatalogTracker. Any
// regionserver should do. Use TESTUTIL.getConfiguration rather than
// the conf from the master; the conf will already have an HConnection
@ -208,7 +207,7 @@ public class TestMasterNoCluster {
HConnection connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
return new CatalogTracker(zk, conf, connection, abortable);
return new CatalogTracker(getZooKeeper(), getConfiguration(), connection, this);
}
@Override
@ -219,17 +218,24 @@ public class TestMasterNoCluster {
try {
// Wait till master is up ready for RPCs.
while (!master.isRpcServerOpen()) Threads.sleep(10);
while (!master.serviceStarted) Threads.sleep(10);
// Fake master that there are regionservers out there. Report in.
for (int i = 0; i < sns.length; i++) {
RegionServerReportRequest.Builder request = RegionServerReportRequest.newBuilder();;
ServerName sn = ServerName.parseVersionedServerName(sns[i].getVersionedBytes());
request.setServer(ProtobufUtil.toServerName(sn));
request.setLoad(ServerLoad.EMPTY_SERVERLOAD.obtainServerLoadPB());
master.regionServerReport(null, request.build());
master.getMasterRpcServices().regionServerReport(null, request.build());
}
ZooKeeperWatcher zkw = master.getZooKeeper();
// Master should now come up.
while (!master.isInitialized()) {Threads.sleep(10);}
while (!master.isInitialized()) {
// Fake meta is closed on rs0, try several times in case the event is lost
// due to race with HMaster#assignMeta
ZKAssign.transitionNodeClosed(zkw,
HRegionInfo.FIRST_META_REGIONINFO, sn0, -1);
Threads.sleep(100);
}
assertTrue(master.isInitialized());
} finally {
rs0.stop("Test is done");
@ -240,120 +246,6 @@ public class TestMasterNoCluster {
}
}
/**
* Test starting master getting it up post initialized state using mocks.
* @throws IOException
* @throws KeeperException
* @throws InterruptedException
* @throws DeserializationException
* @throws ServiceException
*/
@Test (timeout=60000)
public void testCatalogDeploys()
throws Exception {
final Configuration conf = TESTUTIL.getConfiguration();
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, 1);
final long now = System.currentTimeMillis();
// Name for our single mocked up regionserver.
final ServerName sn = ServerName.valueOf("0.example.org", 0, now);
// Here is our mocked up regionserver. Create it now. Need it setting up
// master next.
final MockRegionServer rs0 = new MockRegionServer(conf, sn);
// Create master. Subclass to override a few methods so we can insert mocks
// and get notification on transitions. We need to fake out any rpcs the
// master does opening/closing regions. Also need to fake out the address
// of the 'remote' mocked up regionservers.
HMaster master = new HMaster(conf) {
InetAddress getRemoteInetAddress(final int port, final long serverStartCode)
throws UnknownHostException {
// Interject an unchecked, nonsense InetAddress; i.e. no resolve.
return InetAddress.getByAddress(rs0.getServerName().getHostname(),
new byte [] {10, 0, 0, 0});
}
@Override
ServerManager createServerManager(Server master, MasterServices services)
throws IOException {
ServerManager sm = super.createServerManager(master, services);
// Spy on the created servermanager
ServerManager spy = Mockito.spy(sm);
// Fake a successful open.
Mockito.doReturn(RegionOpeningState.OPENED).when(spy).
sendRegionOpen((ServerName)Mockito.any(), (HRegionInfo)Mockito.any(),
Mockito.anyInt(), Mockito.anyListOf(ServerName.class));
return spy;
}
@Override
CatalogTracker createCatalogTracker(ZooKeeperWatcher zk,
Configuration conf, Abortable abortable)
throws IOException {
// Insert a mock for the connection used by the CatalogTracker. Use
// TESTUTIL.getConfiguration rather than the conf from the master; the
// conf will already have an HConnection associate so the below mocking
// of a connection will fail.
HConnection connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
return new CatalogTracker(zk, conf, connection, abortable);
}
@Override
void initNamespace() {
}
};
master.start();
LOG.info("Master has started");
try {
// Wait till master is up ready for RPCs.
while (!master.isRpcServerOpen()) Threads.sleep(10);
LOG.info("RpcServerOpen has started");
// Fake master that there is a regionserver out there. Report in.
RegionServerStartupRequest.Builder request = RegionServerStartupRequest.newBuilder();
request.setPort(rs0.getServerName().getPort());
request.setServerStartCode(rs0.getServerName().getStartcode());
request.setServerCurrentTime(now);
RegionServerStartupResponse result =
master.regionServerStartup(null, request.build());
String rshostname = new String();
for (NameStringPair e : result.getMapEntriesList()) {
if (e.getName().toString().equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
rshostname = e.getValue();
}
}
// Assert hostname is as expected.
assertEquals(rs0.getServerName().getHostname(), rshostname);
// Now master knows there is at least one regionserver checked in and so
// it'll wait a while to see if more and when none, will assign meta
// to this single server. Will do an rpc open but we've
// mocked it above in our master override to return 'success'. As part of
// region open, master will have set an unassigned znode for the region up
// into zk for the regionserver to transition. Lets do that now to
// complete fake of a successful open.
Mocking.fakeRegionServerRegionOpenInZK(master, rs0.getZooKeeper(),
rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
LOG.info("fakeRegionServerRegionOpenInZK has started");
// Need to set meta location as r0. Usually the regionserver does this
// when its figured it just opened the meta region by setting the meta
// location up into zk. Since we're mocking regionserver, need to do this
// ourselves.
MetaRegionTracker.setMetaLocation(rs0.getZooKeeper(), rs0.getServerName());
// Master should now come up.
while (!master.isInitialized()) {Threads.sleep(10);}
assertTrue(master.isInitialized());
} finally {
rs0.stop("Test is done");
master.stopMaster();
master.join();
}
}
@Test
public void testNotPullingDeadRegionServerFromZK()
throws IOException, KeeperException, InterruptedException {
@ -384,9 +276,7 @@ public class TestMasterNoCluster {
}
@Override
CatalogTracker createCatalogTracker(ZooKeeperWatcher zk,
Configuration conf, Abortable abortable)
throws IOException {
protected CatalogTracker createCatalogTracker() throws IOException {
// Insert a mock for the connection used by the CatalogTracker. Any
// regionserver should do. Use TESTUTIL.getConfiguration rather than
// the conf from the master; the conf will already have an HConnection
@ -394,7 +284,7 @@ public class TestMasterNoCluster {
HConnection connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
return new CatalogTracker(zk, conf, connection, abortable);
return new CatalogTracker(getZooKeeper(), getConfiguration(), connection, this);
}
@Override

View File

@ -21,21 +21,21 @@ package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
import java.util.NavigableSet;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -80,7 +80,7 @@ public class TestMasterRestartAfterDisablingTable {
log("Disabling table\n");
TEST_UTIL.getHBaseAdmin().disableTable(table);
NavigableSet<String> regions = getAllOnlineRegions(cluster);
NavigableSet<String> regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
assertEquals(
"The number of regions for the table tableRestart should be 0 and only"
+ "the catalog and namespace tables should be present.", 2, regions.size());
@ -108,10 +108,10 @@ public class TestMasterRestartAfterDisablingTable {
log("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
log("Verifying there are " + numRegions + " assigned on cluster\n");
regions = getAllOnlineRegions(cluster);
assertEquals(
"The assigned regions were not onlined after master switch except for the catalog and namespace tables.",
6, regions.size());
regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
assertEquals("The assigned regions were not onlined after master"
+ " switch except for the catalog and namespace tables.",
6, regions.size());
assertTrue("The table should be in enabled state", cluster.getMaster()
.getAssignmentManager().getZKTable()
.isEnabledTable(TableName.valueOf("tableRestart")));
@ -128,17 +128,5 @@ public class TestMasterRestartAfterDisablingTable {
ZKAssign.blockUntilNoRIT(zkw);
master.assignmentManager.waitUntilNoRegionsInTransition(60000);
}
private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
throws IOException {
NavigableSet<String> online = new TreeSet<String>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
for (HRegionInfo region : ProtobufUtil.getOnlineRegions(rst.getRegionServer())) {
online.add(region.getRegionNameAsString());
}
}
return online;
}
}

View File

@ -25,18 +25,19 @@ import static org.junit.Assert.assertTrue;
import java.util.List;
import org.apache.hadoop.hbase.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(LargeTests.class)
public class TestMasterShutdown {
private static final Log LOG = LogFactory.getLog(TestMasterShutdown.class);
/**
* Simple test of shutdown.
* <p>
@ -100,6 +101,7 @@ public class TestMasterShutdown {
// Create config to use for this cluster
Configuration conf = HBaseConfiguration.create();
conf.setInt("hbase.ipc.client.failed.servers.expiry", 200);
// Start the cluster
final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);

View File

@ -35,6 +35,8 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
import org.apache.hadoop.hbase.regionserver.MetricsRegionServerWrapperStub;
import org.apache.hadoop.hbase.tmpl.master.AssignmentManagerStatusTmpl;
import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
import org.junit.Before;
@ -94,13 +96,17 @@ public class TestMasterStatusServlet {
// Fake ZKW
ZooKeeperWatcher zkw = Mockito.mock(ZooKeeperWatcher.class);
Mockito.doReturn("fakequorum").when(zkw).getQuorum();
Mockito.doReturn(zkw).when(master).getZooKeeperWatcher();
Mockito.doReturn(zkw).when(master).getZooKeeper();
// Fake MasterAddressTracker
MasterAddressTracker tracker = Mockito.mock(MasterAddressTracker.class);
Mockito.doReturn(tracker).when(master).getMasterAddressTracker();
Mockito.doReturn(FAKE_HOST).when(tracker).getMasterAddress();
MetricsRegionServer rms = Mockito.mock(MetricsRegionServer.class);
Mockito.doReturn(new MetricsRegionServerWrapperStub()).when(rms).getRegionServerWrapper();
Mockito.doReturn(rms).when(master).getRegionServerMetrics();
// Mock admin
admin = Mockito.mock(HBaseAdmin.class);
}

View File

@ -18,7 +18,7 @@
*/
package org.apache.hadoop.hbase.master;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.List;
@ -29,7 +29,12 @@ import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
@ -86,7 +91,7 @@ public class TestRollingRestart {
TEST_UTIL.getHBaseAdmin().disableTable(table);
log("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
NavigableSet<String> regions = getAllOnlineRegions(cluster);
NavigableSet<String> regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
log("Verifying only catalog and namespace regions are assigned\n");
if (regions.size() != 2) {
for (String oregion : regions) log("Region still online: " + oregion);
@ -97,7 +102,7 @@ public class TestRollingRestart {
log("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
log("Verifying there are " + numRegions + " assigned on cluster\n");
regions = getAllOnlineRegions(cluster);
regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
assertRegionsAssigned(cluster, regions);
assertEquals(expectedNumRS, cluster.getRegionServerThreads().size());
@ -184,102 +189,6 @@ public class TestRollingRestart {
Thread.sleep(1000);
assertRegionsAssigned(cluster, regions);
// Bring the RS hosting hbase:meta down
RegionServerThread metaServer = getServerHostingMeta(cluster);
log("Stopping server hosting hbase:meta #1");
metaServer.getRegionServer().stop("Stopping hbase:meta server");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down #1");
expectedNumRS--;
log("Waiting for meta server #1 RS shutdown to be handled by master");
waitForRSShutdownToStartAndFinish(activeMaster,
metaServer.getRegionServer().getServerName());
log("Waiting for no more RIT");
long start = System.currentTimeMillis();
do {
blockUntilNoRIT(zkw, master);
} while (getNumberOfOnlineRegions(cluster) < numRegions
&& System.currentTimeMillis()-start < 60000);
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
assertEquals(expectedNumRS, cluster.getRegionServerThreads().size());
// Kill off the server hosting hbase:meta again
metaServer = getServerHostingMeta(cluster);
log("Stopping server hosting hbase:meta #2");
metaServer.getRegionServer().stop("Stopping hbase:meta server");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down");
expectedNumRS--;
log("Waiting for RS shutdown to be handled by master");
waitForRSShutdownToStartAndFinish(activeMaster,
metaServer.getRegionServer().getServerName());
log("RS shutdown done, waiting for no more RIT");
blockUntilNoRIT(zkw, master);
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
assertEquals(expectedNumRS, cluster.getRegionServerThreads().size());
// Start 3 RS again
cluster.startRegionServer().waitForServerOnline();
cluster.startRegionServer().waitForServerOnline();
cluster.startRegionServer().waitForServerOnline();
Thread.sleep(1000);
log("Waiting for no more RIT");
blockUntilNoRIT(zkw, master);
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
// Shutdown server hosting META
metaServer = getServerHostingMeta(cluster);
log("Stopping server hosting hbase:meta (1 of 3)");
metaServer.getRegionServer().stop("Stopping hbase:meta server");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down (1 of 3)");
log("Waiting for RS shutdown to be handled by master");
waitForRSShutdownToStartAndFinish(activeMaster,
metaServer.getRegionServer().getServerName());
log("RS shutdown done, waiting for no more RIT");
blockUntilNoRIT(zkw, master);
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
// Shutdown server hosting hbase:meta again
metaServer = getServerHostingMeta(cluster);
log("Stopping server hosting hbase:meta (2 of 3)");
metaServer.getRegionServer().stop("Stopping hbase:meta server");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down (2 of 3)");
log("Waiting for RS shutdown to be handled by master");
waitForRSShutdownToStartAndFinish(activeMaster,
metaServer.getRegionServer().getServerName());
log("RS shutdown done, waiting for no more RIT");
blockUntilNoRIT(zkw, master);
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
// Shutdown server hosting hbase:meta again
metaServer = getServerHostingMeta(cluster);
log("Stopping server hosting hbase:meta (3 of 3)");
metaServer.getRegionServer().stop("Stopping hbase:meta server");
cluster.hbaseCluster.waitOnRegionServer(metaServer);
log("Meta server down (3 of 3)");
log("Waiting for RS shutdown to be handled by master");
waitForRSShutdownToStartAndFinish(activeMaster,
metaServer.getRegionServer().getServerName());
log("RS shutdown done, waiting for no more RIT");
blockUntilNoRIT(zkw, master);
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
if (cluster.getRegionServerThreads().size() != 1) {
log("Online regionservers:");
for (RegionServerThread rst : cluster.getRegionServerThreads()) {
log("RS: " + rst.getRegionServer().getServerName());
}
}
assertEquals(2, cluster.getRegionServerThreads().size());
// TODO: Bring random 3 of 4 RS down at the same time
ht.close();
@ -314,26 +223,14 @@ public class TestRollingRestart {
LOG.debug("\n\nTRR: " + msg + "\n");
}
private RegionServerThread getServerHostingMeta(MiniHBaseCluster cluster)
throws IOException {
return getServerHosting(cluster, HRegionInfo.FIRST_META_REGIONINFO);
}
private RegionServerThread getServerHosting(MiniHBaseCluster cluster,
HRegionInfo region) throws IOException {
for (RegionServerThread rst : cluster.getRegionServerThreads()) {
if (ProtobufUtil.getOnlineRegions(rst.getRegionServer()).contains(region)) {
return rst;
}
}
return null;
}
private int getNumberOfOnlineRegions(MiniHBaseCluster cluster) {
int numFound = 0;
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
numFound += rst.getRegionServer().getNumberOfOnlineRegions();
}
for (MasterThread mt : cluster.getMasterThreads()) {
numFound += mt.getMaster().getNumberOfOnlineRegions();
}
return numFound;
}
@ -343,7 +240,8 @@ public class TestRollingRestart {
if (expectedRegions.size() > numFound) {
log("Expected to find " + expectedRegions.size() + " but only found"
+ " " + numFound);
NavigableSet<String> foundRegions = getAllOnlineRegions(cluster);
NavigableSet<String> foundRegions =
HBaseTestingUtility.getAllOnlineRegions(cluster);
for (String region : expectedRegions) {
if (!foundRegions.contains(region)) {
log("Missing region: " + region);
@ -364,23 +262,13 @@ public class TestRollingRestart {
}
}
private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
throws IOException {
NavigableSet<String> online = new TreeSet<String>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
for (HRegionInfo region : ProtobufUtil.getOnlineRegions(rst.getRegionServer())) {
online.add(region.getRegionNameAsString());
}
}
return online;
}
private NavigableSet<String> getDoubleAssignedRegions(
MiniHBaseCluster cluster) throws IOException {
NavigableSet<String> online = new TreeSet<String>();
NavigableSet<String> doubled = new TreeSet<String>();
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
for (HRegionInfo region : ProtobufUtil.getOnlineRegions(rst.getRegionServer())) {
for (HRegionInfo region : ProtobufUtil.getOnlineRegions(
rst.getRegionServer().getRSRpcServices())) {
if(!online.add(region.getRegionNameAsString())) {
doubled.add(region.getRegionNameAsString());
}

View File

@ -107,7 +107,7 @@ public class TestTableLockManager {
conf.setInt(TableLockManager.TABLE_WRITE_LOCK_TIMEOUT_MS, 3000);
prepareMiniCluster();
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
master.getCoprocessorHost().load(TestLockTimeoutExceptionMasterObserver.class,
master.getMasterCoprocessorHost().load(TestLockTimeoutExceptionMasterObserver.class,
0, TEST_UTIL.getConfiguration());
ExecutorService executor = Executors.newSingleThreadExecutor();
@ -159,7 +159,7 @@ public class TestTableLockManager {
// thread, send a request to disable, and then delete a table.
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
master.getCoprocessorHost().load(TestAlterAndDisableMasterObserver.class,
master.getMasterCoprocessorHost().load(TestAlterAndDisableMasterObserver.class,
0, TEST_UTIL.getConfiguration());
ExecutorService executor = Executors.newFixedThreadPool(2);

View File

@ -111,7 +111,8 @@ public class TestZKBasedOpenCloseRegion {
int rsIdx = 0;
HRegionServer regionServer =
TEST_UTIL.getHBaseCluster().getRegionServer(rsIdx);
HRegionInfo hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(regionServer));
HRegionInfo hri = getNonMetaRegion(
ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices()));
LOG.debug("Asking RS to close region " + hri.getRegionNameAsString());
LOG.info("Unassign " + hri.getRegionNameAsString());
@ -154,7 +155,7 @@ public class TestZKBasedOpenCloseRegion {
cluster.getLiveRegionServerThreads().get(0).getRegionServer();
HRegionServer hr1 =
cluster.getLiveRegionServerThreads().get(1).getRegionServer();
HRegionInfo hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr0));
HRegionInfo hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr0.getRSRpcServices()));
// fake that hr1 is processing the region
hr1.getRegionsInTransitionInRS().putIfAbsent(hri.getEncodedNameAsBytes(), true);
@ -170,7 +171,7 @@ public class TestZKBasedOpenCloseRegion {
hr1.getRegionsInTransitionInRS().remove(hri.getEncodedNameAsBytes());
// now try moving a region when there is no region in transition.
hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr1));
hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr1.getRSRpcServices()));
TEST_UTIL.getHBaseAdmin().move(hri.getEncodedNameAsBytes(),
Bytes.toBytes(hr0.getServerName().toString()));
@ -192,7 +193,8 @@ public class TestZKBasedOpenCloseRegion {
int rsIdx = 0;
HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(rsIdx);
HRegionInfo hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(regionServer));
HRegionInfo hri = getNonMetaRegion(
ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices()));
LOG.debug("Asking RS to close region " + hri.getRegionNameAsString());
cluster.getMaster().assignmentManager.unassign(hri);
@ -230,7 +232,8 @@ public class TestZKBasedOpenCloseRegion {
Whitebox.setInternalState(regionServer, "tableDescriptors", htd);
Mockito.doThrow(new IOException()).when(htd).get((TableName) Mockito.any());
try {
ProtobufUtil.openRegion(regionServer, regionServer.getServerName(), REGIONINFO);
ProtobufUtil.openRegion(regionServer.getRSRpcServices(),
regionServer.getServerName(), REGIONINFO);
fail("It should throw IOException ");
} catch (IOException e) {
}

View File

@ -164,7 +164,7 @@ public class BalancerTestBase {
}
protected BaseLoadBalancer.Cluster mockCluster(int[] mockCluster) {
return new BaseLoadBalancer.Cluster(mockClusterServers(mockCluster, -1), null, null);
return new BaseLoadBalancer.Cluster(null, mockClusterServers(mockCluster, -1), null, null);
}
protected Map<ServerName, List<HRegionInfo>> mockClusterServers(int[] mockCluster, int numTables) {

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.master.balancer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@ -39,11 +40,13 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import com.google.common.collect.Lists;
@ -52,6 +55,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
private static LoadBalancer loadBalancer;
private static final Log LOG = LogFactory.getLog(TestStochasticLoadBalancer.class);
private static final ServerName master = ServerName.valueOf("fake-master", 0, 1L);
int[][] regionsAndServersMocks = new int[][] {
// { num regions, num servers }
@ -65,6 +69,9 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
Configuration conf = HBaseConfiguration.create();
loadBalancer = new MockBalancer();
loadBalancer.setConf(conf);
MasterServices st = Mockito.mock(MasterServices.class);
Mockito.when(st.getServerName()).thenReturn(master);
loadBalancer.setMasterServices(st);
}
public static class MockBalancer extends BaseLoadBalancer {
@ -85,6 +92,17 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
*/
@Test
public void testImmediateAssignment() throws Exception {
List<ServerName> tmp = getListOfServerNames(randomServers(1, 0));
tmp.add(master);
ServerName sn = loadBalancer.randomAssignment(HRegionInfo.FIRST_META_REGIONINFO, tmp);
assertEquals(master, sn);
HRegionInfo hri = randomRegions(1, -1).get(0);
sn = loadBalancer.randomAssignment(hri, tmp);
assertNotEquals(master, sn);
tmp = new ArrayList<ServerName>();
tmp.add(master);
sn = loadBalancer.randomAssignment(hri, tmp);
assertEquals(master, sn);
for (int[] mock : regionsAndServersMocks) {
LOG.debug("testImmediateAssignment with " + mock[0] + " regions and " + mock[1] + " servers");
List<HRegionInfo> regions = randomRegions(mock[0]);
@ -120,6 +138,18 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
*/
@Test
public void testBulkAssignment() throws Exception {
List<ServerName> tmp = getListOfServerNames(randomServers(5, 0));
List<HRegionInfo> hris = randomRegions(20);
hris.add(HRegionInfo.FIRST_META_REGIONINFO);
tmp.add(master);
Map<ServerName, List<HRegionInfo>> plans = loadBalancer.roundRobinAssignment(hris, tmp);
assertTrue(plans.get(master).contains(HRegionInfo.FIRST_META_REGIONINFO));
assertEquals(1, plans.get(master).size());
int totalRegion = 0;
for (List<HRegionInfo> regions: plans.values()) {
totalRegion += regions.size();
}
assertEquals(hris.size(), totalRegion);
for (int[] mock : regionsAndServersMocks) {
LOG.debug("testBulkAssignment with " + mock[0] + " regions and " + mock[1] + " servers");
List<HRegionInfo> regions = randomRegions(mock[0]);
@ -256,7 +286,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
assignRegions(regions, oldServers, clusterState);
// should not throw exception:
BaseLoadBalancer.Cluster cluster = new Cluster(clusterState, null, null);
BaseLoadBalancer.Cluster cluster = new Cluster(null, clusterState, null, null);
assertEquals(101 + 9, cluster.numRegions);
assertEquals(10, cluster.numServers); // only 10 servers because they share the same host + port
}
@ -298,7 +328,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
when(locationFinder.getTopBlockLocations(regions.get(43))).thenReturn(
Lists.newArrayList(ServerName.valueOf("foo", 0, 0))); // this server does not exists in clusterStatus
BaseLoadBalancer.Cluster cluster = new Cluster(clusterState, null, locationFinder);
BaseLoadBalancer.Cluster cluster = new Cluster(null, clusterState, null, locationFinder);
int r0 = ArrayUtils.indexOf(cluster.regions, regions.get(0)); // this is ok, it is just a test
int r1 = ArrayUtils.indexOf(cluster.regions, regions.get(1));

View File

@ -30,7 +30,6 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -79,7 +78,6 @@ public class TestSnapshotFromMaster {
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final int NUM_RS = 2;
private static Path rootDir;
private static Path snapshots;
private static FileSystem fs;
private static HMaster master;
@ -101,7 +99,6 @@ public class TestSnapshotFromMaster {
fs = UTIL.getDFSCluster().getFileSystem();
master = UTIL.getMiniHBaseCluster().getMaster();
rootDir = master.getMasterFileSystem().getRootDir();
snapshots = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
archiveDir = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
}
@ -194,7 +191,8 @@ public class TestSnapshotFromMaster {
// then do the lookup for the snapshot that it is done
builder.setSnapshot(desc);
IsSnapshotDoneResponse response = master.isSnapshotDone(null, builder.build());
IsSnapshotDoneResponse response =
master.getMasterRpcServices().isSnapshotDone(null, builder.build());
assertTrue("Snapshot didn't complete when it should have.", response.getDone());
// now try the case where we are looking for a snapshot we didn't take
@ -209,7 +207,7 @@ public class TestSnapshotFromMaster {
SnapshotDescriptionUtils.writeSnapshotInfo(desc, snapshotDir, fs);
builder.setSnapshot(desc);
response = master.isSnapshotDone(null, builder.build());
response = master.getMasterRpcServices().isSnapshotDone(null, builder.build());
assertTrue("Completed, on-disk snapshot not found", response.getDone());
}
@ -217,7 +215,8 @@ public class TestSnapshotFromMaster {
public void testGetCompletedSnapshots() throws Exception {
// first check when there are no snapshots
GetCompletedSnapshotsRequest request = GetCompletedSnapshotsRequest.newBuilder().build();
GetCompletedSnapshotsResponse response = master.getCompletedSnapshots(null, request);
GetCompletedSnapshotsResponse response =
master.getMasterRpcServices().getCompletedSnapshots(null, request);
assertEquals("Found unexpected number of snapshots", 0, response.getSnapshotsCount());
// write one snapshot to the fs
@ -227,7 +226,7 @@ public class TestSnapshotFromMaster {
SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
// check that we get one snapshot
response = master.getCompletedSnapshots(null, request);
response = master.getMasterRpcServices().getCompletedSnapshots(null, request);
assertEquals("Found unexpected number of snapshots", 1, response.getSnapshotsCount());
List<SnapshotDescription> snapshots = response.getSnapshotsList();
List<SnapshotDescription> expected = Lists.newArrayList(snapshot);
@ -241,7 +240,7 @@ public class TestSnapshotFromMaster {
expected.add(snapshot);
// check that we get one snapshot
response = master.getCompletedSnapshots(null, request);
response = master.getMasterRpcServices().getCompletedSnapshots(null, request);
assertEquals("Found unexpected number of snapshots", 2, response.getSnapshotsCount());
snapshots = response.getSnapshotsList();
assertEquals("Returned snapshots don't match created snapshots", expected, snapshots);
@ -256,7 +255,7 @@ public class TestSnapshotFromMaster {
DeleteSnapshotRequest request = DeleteSnapshotRequest.newBuilder().setSnapshot(snapshot)
.build();
try {
master.deleteSnapshot(null, request);
master.getMasterRpcServices().deleteSnapshot(null, request);
fail("Master didn't throw exception when attempting to delete snapshot that doesn't exist");
} catch (ServiceException e) {
LOG.debug("Correctly failed delete of non-existant snapshot:" + e.getMessage());
@ -267,7 +266,7 @@ public class TestSnapshotFromMaster {
SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
// then delete the existing snapshot,which shouldn't cause an exception to be thrown
master.deleteSnapshot(null, request);
master.getMasterRpcServices().deleteSnapshot(null, request);
}
/**

View File

@ -50,7 +50,7 @@ public class OOMERegionServer extends HRegionServer {
try {
MutateRequest request =
RequestConverter.buildMutateRequest(regionName, put);
super.mutate(null, request);
rpcServices.mutate(null, request);
for (int i = 0; i < 30; i++) {
// Add the batch update 30 times to bring on the OOME faster.
this.retainer.add(put);

View File

@ -23,8 +23,6 @@ import static org.junit.Assert.assertNotNull;
import java.util.UUID;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@ -48,14 +46,10 @@ import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestClusterId {
private static final Log LOG =
LogFactory.getLog(TestClusterId.class.getName());
private final HBaseTestingUtility TEST_UTIL =
new HBaseTestingUtility();
private JVMClusterUtil.RegionServerThread rst;
private JVMClusterUtil.MasterThread mst;
@Before
public void setUp() throws Exception {
@ -113,7 +107,7 @@ public class TestClusterId {
}
TEST_UTIL.startMiniHBaseCluster(1, 1);
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
assertEquals(1, master.getServerManager().getOnlineServersList().size());
assertEquals(2, master.getServerManager().getOnlineServersList().size());
}
}

View File

@ -158,11 +158,12 @@ public class TestEndToEndSplitTransaction {
// get and scan should now succeed without exception
ClientProtos.GetRequest request =
RequestConverter.buildGetRequest(regionName, new Get(row));
server.get(null, request);
server.getRSRpcServices().get(null, request);
ScanRequest scanRequest = RequestConverter.buildScanRequest(
regionName, new Scan(row), 1, true);
try {
server.scan(new PayloadCarryingRpcController(), scanRequest);
server.getRSRpcServices().scan(
new PayloadCarryingRpcController(), scanRequest);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}

View File

@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
@ -43,6 +42,7 @@ import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import com.google.protobuf.ByteString;
import com.google.protobuf.HBaseZeroCopyByteString;
/**
* Tests that verify certain RPCs get a higher QoS.
@ -55,13 +55,14 @@ public class TestPriorityRpc {
@Before
public void setup() {
Configuration conf = HBaseConfiguration.create();
conf.setBoolean("hbase.testing.nocluster", true); // No need to do ZK
regionServer = HRegionServer.constructRegionServer(HRegionServer.class, conf);
priority = regionServer.getPriority();
priority = regionServer.rpcServices.getPriority();
}
@Test
public void testQosFunctionForMeta() throws IOException {
priority = regionServer.getPriority();
priority = regionServer.rpcServices.getPriority();
RequestHeader.Builder headerBuilder = RequestHeader.newBuilder();
//create a rpc request that has references to hbase:meta region and also
//uses one of the known argument classes (known argument classes are
@ -82,8 +83,10 @@ public class TestPriorityRpc {
RequestHeader header = headerBuilder.build();
HRegion mockRegion = Mockito.mock(HRegion.class);
HRegionServer mockRS = Mockito.mock(HRegionServer.class);
RSRpcServices mockRpc = Mockito.mock(RSRpcServices.class);
Mockito.when(mockRS.getRSRpcServices()).thenReturn(mockRpc);
HRegionInfo mockRegionInfo = Mockito.mock(HRegionInfo.class);
Mockito.when(mockRS.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRpc.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(true);
// Presume type.
@ -100,7 +103,7 @@ public class TestPriorityRpc {
RequestHeader.Builder headerBuilder = RequestHeader.newBuilder();
headerBuilder.setMethodName("foo");
RequestHeader header = headerBuilder.build();
PriorityFunction qosFunc = regionServer.getPriority();
PriorityFunction qosFunc = regionServer.rpcServices.getPriority();
assertEquals(HConstants.NORMAL_QOS, qosFunc.getPriority(header, null));
}
@ -115,8 +118,10 @@ public class TestPriorityRpc {
ScanRequest scanRequest = scanBuilder.build();
HRegion mockRegion = Mockito.mock(HRegion.class);
HRegionServer mockRS = Mockito.mock(HRegionServer.class);
RSRpcServices mockRpc = Mockito.mock(RSRpcServices.class);
Mockito.when(mockRS.getRSRpcServices()).thenReturn(mockRpc);
HRegionInfo mockRegionInfo = Mockito.mock(HRegionInfo.class);
Mockito.when(mockRS.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRpc.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(false);
// Presume type.
@ -130,9 +135,9 @@ public class TestPriorityRpc {
scanRequest = scanBuilder.build();
//mock out a high priority type handling and see the QoS returned
RegionScanner mockRegionScanner = Mockito.mock(RegionScanner.class);
Mockito.when(mockRS.getScanner(12345)).thenReturn(mockRegionScanner);
Mockito.when(mockRpc.getScanner(12345)).thenReturn(mockRegionScanner);
Mockito.when(mockRegionScanner.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRS.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRpc.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(true);

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