locateRegions(final TableName tableName,
final boolean useCache,
final boolean offlined) throws IOException;
@@ -186,12 +203,12 @@ public interface ClusterConnection extends HConnection {
* @param useCache Should we use the cache to retrieve the region information.
* @param retry do we retry
* @return region locations for this row.
- * @throws IOException
+ * @throws IOException if IO failure occurs
*/
RegionLocations locateRegion(TableName tableName,
byte[] row, boolean useCache, boolean retry) throws IOException;
- /**
+ /**
*
* @param tableName table to get regions of
* @param row the row
@@ -199,15 +216,14 @@ public interface ClusterConnection extends HConnection {
* @param retry do we retry
* @param replicaId the replicaId for the region
* @return region locations for this row.
- * @throws IOException
+ * @throws IOException if IO failure occurs
*/
- RegionLocations locateRegion(TableName tableName, byte[] row, boolean useCache, boolean retry,
+ RegionLocations locateRegion(TableName tableName, byte[] row, boolean useCache, boolean retry,
int replicaId) throws IOException;
/**
* Returns a {@link MasterKeepAliveConnection} to the active master
*/
- @Override
MasterService.BlockingInterface getMaster() throws IOException;
@@ -217,7 +233,6 @@ public interface ClusterConnection extends HConnection {
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
*/
- @Override
AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
/**
@@ -229,7 +244,6 @@ public interface ClusterConnection extends HConnection {
* @throws IOException if a remote or network exception occurs
*
*/
- @Override
ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
/**
@@ -240,7 +254,6 @@ public interface ClusterConnection extends HConnection {
* @return Location of row.
* @throws IOException if a remote or network exception occurs
*/
- @Override
HRegionLocation getRegionLocation(TableName tableName, byte [] row,
boolean reload)
throws IOException;
@@ -249,34 +262,30 @@ public interface ClusterConnection extends HConnection {
* Clear any caches that pertain to server name sn.
* @param sn A server name
*/
- @Override
void clearCaches(final ServerName sn);
/**
* This function allows HBaseAdmin and potentially others to get a shared MasterService
* connection.
* @return The shared instance. Never returns null.
- * @throws MasterNotRunningException
+ * @throws MasterNotRunningException if master is not running
* @deprecated Since 0.96.0
*/
- @Override
@Deprecated
MasterKeepAliveConnection getKeepAliveMasterService()
throws MasterNotRunningException;
/**
- * @param serverName
+ * @param serverName of server to check
* @return true if the server is known as dead, false otherwise.
- * @deprecated internal method, do not use thru HConnection */
- @Override
+ * @deprecated internal method, do not use thru ClusterConnection */
@Deprecated
boolean isDeadServer(ServerName serverName);
/**
- * @return Nonce generator for this HConnection; may be null if disabled in configuration.
+ * @return Nonce generator for this ClusterConnection; may be null if disabled in configuration.
*/
- @Override
- public NonceGenerator getNonceGenerator();
+ NonceGenerator getNonceGenerator();
/**
* @return Default AsyncProcess associated with this connection.
@@ -287,7 +296,7 @@ public interface ClusterConnection extends HConnection {
* Returns a new RpcRetryingCallerFactory from the given {@link Configuration}.
* This RpcRetryingCallerFactory lets the users create {@link RpcRetryingCaller}s which can be
* intercepted with the configured {@link RetryingCallerInterceptor}
- * @param conf
+ * @param conf configuration
* @return RpcRetryingCallerFactory
*/
RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf);
@@ -320,11 +329,17 @@ public interface ClusterConnection extends HConnection {
/**
* @return the MetricsConnection instance associated with this connection.
*/
- public MetricsConnection getConnectionMetrics();
+ MetricsConnection getConnectionMetrics();
/**
* @return true when this connection uses a {@link org.apache.hadoop.hbase.codec.Codec} and so
* supports cell blocks.
*/
boolean hasCellBlockSupport();
+
+ /**
+ * @return the number of region servers that are currently running
+ * @throws IOException if a remote or network exception occurs
+ */
+ int getCurrentNrHRS() throws IOException;
}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
index a3f6fe6..b979c6a 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
@@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
* thread will obtain its own Table instance. Caching or pooling of {@link Table} and {@link Admin}
* is not recommended.
*
- * This class replaces {@link HConnection}, which is now deprecated.
* @see ConnectionFactory
* @since 0.99.0
*/
@@ -59,7 +58,7 @@ public interface Connection extends Abortable, Closeable {
* - Only allow new style of interfaces:
* -- All table names are passed as TableName. No more byte[] and string arguments
* -- Most of the classes with names H is deprecated in favor of non-H versions
- * (Table, Connection vs HConnection, etc)
+ * (Table, Connection, etc)
* -- Only real client-facing public methods are allowed
* - Connection should contain only getTable(), getAdmin() kind of general methods.
*/
@@ -123,7 +122,7 @@ public interface Connection extends Abortable, Closeable {
*
* @return a {@link BufferedMutator} for the supplied tableName.
*/
- public BufferedMutator getBufferedMutator(TableName tableName) throws IOException;
+ BufferedMutator getBufferedMutator(TableName tableName) throws IOException;
/**
* Retrieve a {@link BufferedMutator} for performing client-side buffering of writes. The
@@ -134,7 +133,7 @@ public interface Connection extends Abortable, Closeable {
* @param params details on how to instantiate the {@code BufferedMutator}.
* @return a {@link BufferedMutator} for the supplied tableName.
*/
- public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException;
+ BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException;
/**
* Retrieve a RegionLocator implementation to inspect region information on a table. The returned
@@ -151,7 +150,7 @@ public interface Connection extends Abortable, Closeable {
* @param tableName Name of the table who's region is to be examined
* @return A RegionLocator instance
*/
- public RegionLocator getRegionLocator(TableName tableName) throws IOException;
+ RegionLocator getRegionLocator(TableName tableName) throws IOException;
/**
* Retrieve an Admin implementation to administer an HBase cluster.
@@ -167,7 +166,7 @@ public interface Connection extends Abortable, Closeable {
Admin getAdmin() throws IOException;
@Override
- public void close() throws IOException;
+ void close() throws IOException;
/**
* Returns whether the connection is closed or not.
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
index 3e8ca31..a5dbddd 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
@@ -214,7 +214,7 @@ public class ConnectionFactory {
user = provider.getCurrent();
}
- String className = conf.get(HConnection.HBASE_CLIENT_CONNECTION_IMPL,
+ String className = conf.get(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL,
ConnectionImplementation.class.getName());
Class> clazz;
try {
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 429e47d..d93a8b4 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations;
@@ -64,13 +63,11 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcClientFactory;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -98,7 +95,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.KeeperException;
-
/**
* Main implementation of {@link Connection} and {@link ClusterConnection} interfaces.
* Encapsulates connection to zookeeper and regionservers.
@@ -124,8 +120,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
* Once it's set under nonceGeneratorCreateLock, it is never unset or changed.
*/
private static volatile NonceGenerator nonceGenerator = null;
- /** The nonce generator lock. Only taken when creating HConnection, which gets a private copy. */
- private static Object nonceGeneratorCreateLock = new Object();
+ /** The nonce generator lock. Only taken when creating Connection, which gets a private copy. */
+ private static final Object nonceGeneratorCreateLock = new Object();
private final AsyncProcess asyncProcess;
// single tracker per connection
@@ -137,7 +133,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
// package protected for the tests
ClusterStatusListener clusterStatusListener;
-
private final Object metaRegionLock = new Object();
// We have a single lock for master & zk to prevent deadlocks. Having
@@ -162,23 +157,23 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
private final ConnectionConfiguration connectionConfig;
// Client rpc instance.
- private RpcClient rpcClient;
+ private final RpcClient rpcClient;
private final MetaCache metaCache;
private final MetricsConnection metrics;
protected User user;
- private RpcRetryingCallerFactory rpcCallerFactory;
+ private final RpcRetryingCallerFactory rpcCallerFactory;
- private RpcControllerFactory rpcControllerFactory;
+ private final RpcControllerFactory rpcControllerFactory;
private final RetryingCallerInterceptor interceptor;
/**
* Cluster registry of basic info such as clusterid and meta region location.
*/
- Registry registry;
+ Registry registry;
private final ClientBackoffPolicy backoffPolicy;
@@ -279,34 +274,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return ng;
}
-
- @Override
- public HTableInterface getTable(String tableName) throws IOException {
- return getTable(TableName.valueOf(tableName));
- }
-
@Override
- public HTableInterface getTable(byte[] tableName) throws IOException {
- return getTable(TableName.valueOf(tableName));
- }
-
- @Override
- public HTableInterface getTable(TableName tableName) throws IOException {
+ public Table getTable(TableName tableName) throws IOException {
return getTable(tableName, getBatchPool());
}
@Override
- public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException {
- return getTable(TableName.valueOf(tableName), pool);
- }
-
- @Override
- public HTableInterface getTable(byte[] tableName, ExecutorService pool) throws IOException {
- return getTable(TableName.valueOf(tableName), pool);
- }
-
- @Override
- public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException {
+ public Table getTable(TableName tableName, ExecutorService pool) throws IOException {
return new HTable(tableName, this, connectionConfig,
rpcCallerFactory, rpcControllerFactory, pool);
}
@@ -463,7 +437,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
protected String clusterId = null;
protected void retrieveClusterId() {
- if (clusterId != null) return;
+ if (clusterId != null) {
+ return;
+ }
this.clusterId = this.registry.getClusterId();
if (clusterId == null) {
clusterId = HConstants.CLUSTER_ID_DEFAULT;
@@ -519,12 +495,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return reload? relocateRegion(tableName, row): locateRegion(tableName, row);
}
- @Override
- public HRegionLocation getRegionLocation(final byte[] tableName,
- final byte [] row, boolean reload)
- throws IOException {
- return getRegionLocation(TableName.valueOf(tableName), row, reload);
- }
@Override
public boolean isTableEnabled(TableName tableName) throws IOException {
@@ -532,34 +502,16 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
@Override
- public boolean isTableEnabled(byte[] tableName) throws IOException {
- return isTableEnabled(TableName.valueOf(tableName));
- }
-
- @Override
public boolean isTableDisabled(TableName tableName) throws IOException {
return getTableState(tableName).inStates(TableState.State.DISABLED);
}
@Override
- public boolean isTableDisabled(byte[] tableName) throws IOException {
- return isTableDisabled(TableName.valueOf(tableName));
- }
-
- @Override
- public boolean isTableAvailable(final TableName tableName) throws IOException {
- return isTableAvailable(tableName, null);
- }
-
- @Override
- public boolean isTableAvailable(final byte[] tableName) throws IOException {
- return isTableAvailable(TableName.valueOf(tableName));
- }
-
- @Override
public boolean isTableAvailable(final TableName tableName, @Nullable final byte[][] splitKeys)
throws IOException {
- if (this.closed) throw new IOException(toString() + " closed");
+ if (this.closed) {
+ throw new IOException(toString() + " closed");
+ }
try {
if (!isTableEnabled(tableName)) {
LOG.debug("Table " + tableName + " not enabled");
@@ -616,12 +568,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
@Override
- public boolean isTableAvailable(final byte[] tableName, final byte[][] splitKeys)
- throws IOException {
- return isTableAvailable(TableName.valueOf(tableName), splitKeys);
- }
-
- @Override
public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
RegionLocations locations = locateRegion(HRegionInfo.getTable(regionName),
HRegionInfo.getStartKey(regionName), false, true);
@@ -644,12 +590,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
@Override
- public List locateRegions(final byte[] tableName)
- throws IOException {
- return locateRegions(TableName.valueOf(tableName));
- }
-
- @Override
public List locateRegions(final TableName tableName,
final boolean useCache, final boolean offlined) throws IOException {
List regions = MetaTableAccessor
@@ -669,12 +609,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
@Override
- public List locateRegions(final byte[] tableName,
- final boolean useCache, final boolean offlined) throws IOException {
- return locateRegions(TableName.valueOf(tableName), useCache, offlined);
- }
-
- @Override
public HRegionLocation locateRegion(
final TableName tableName, final byte[] row) throws IOException{
RegionLocations locations = locateRegion(tableName, row, true, true);
@@ -682,13 +616,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
@Override
- public HRegionLocation locateRegion(final byte[] tableName,
- final byte [] row)
- throws IOException{
- return locateRegion(TableName.valueOf(tableName), row);
- }
-
- @Override
public HRegionLocation relocateRegion(final TableName tableName,
final byte [] row) throws IOException{
RegionLocations locations = relocateRegion(tableName, row,
@@ -711,12 +638,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
@Override
- public HRegionLocation relocateRegion(final byte[] tableName,
- final byte [] row) throws IOException {
- return relocateRegion(TableName.valueOf(tableName), row);
- }
-
- @Override
public RegionLocations locateRegion(final TableName tableName,
final byte [] row, boolean useCache, boolean retry)
throws IOException {
@@ -727,7 +648,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
public RegionLocations locateRegion(final TableName tableName,
final byte [] row, boolean useCache, boolean retry, int replicaId)
throws IOException {
- if (this.closed) throw new IOException(toString() + " closed");
+ if (this.closed) {
+ throw new IOException(toString() + " closed");
+ }
if (tableName== null || tableName.getName().length == 0) {
throw new IllegalArgumentException(
"table name cannot be null or zero length");
@@ -966,11 +889,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
metaCache.clearCache(tableName);
}
- @Override
- public void clearRegionCache(final byte[] tableName) {
- clearRegionCache(TableName.valueOf(tableName));
- }
-
/**
* Put a newly discovered HRegionLocation into the cache.
* @param tableName The table name.
@@ -993,11 +911,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
* State of the MasterService connection/setup.
*/
static class MasterServiceState {
- HConnection connection;
+ Connection connection;
MasterProtos.MasterService.BlockingInterface stub;
int userCount;
- MasterServiceState(final HConnection connection) {
+ MasterServiceState(final Connection connection) {
super();
this.connection = connection;
}
@@ -1189,7 +1107,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
/**
* Create a stub against the master. Retry if necessary.
* @return A stub to do intf against the master
- * @throws org.apache.hadoop.hbase.MasterNotRunningException
+ * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
*/
Object makeStub() throws IOException {
// The lock must be at the beginning to prevent multiple master creations
@@ -1245,26 +1163,18 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
@Override
public AdminProtos.AdminService.BlockingInterface getAdmin(final ServerName serverName)
throws IOException {
- return getAdmin(serverName, false);
- }
-
- @Override
- // Nothing is done w/ the 'master' parameter. It is ignored.
- public AdminProtos.AdminService.BlockingInterface getAdmin(final ServerName serverName,
- final boolean master)
- throws IOException {
if (isDeadServer(serverName)) {
throw new RegionServerStoppedException(serverName + " is dead.");
}
String key = getStubKey(AdminProtos.AdminService.BlockingInterface.class.getName(),
- serverName.getHostname(), serverName.getPort(), this.hostnamesCanChange);
+ serverName.getHostname(), serverName.getPort(), this.hostnamesCanChange);
this.connectionLock.putIfAbsent(key, key);
- AdminProtos.AdminService.BlockingInterface stub = null;
+ AdminProtos.AdminService.BlockingInterface stub;
synchronized (this.connectionLock.get(key)) {
stub = (AdminProtos.AdminService.BlockingInterface)this.stubs.get(key);
if (stub == null) {
BlockingRpcChannel channel =
- this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout);
+ this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout);
stub = AdminProtos.AdminService.newBlockingStub(channel);
this.stubs.put(key, stub);
}
@@ -1798,7 +1708,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
void releaseMaster(MasterServiceState mss) {
- if (mss.getStub() == null) return;
+ if (mss.getStub() == null) {
+ return;
+ }
synchronized (masterAndZKLock) {
--mss.userCount;
}
@@ -1833,20 +1745,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
metaCache.clearCache(location);
}
- @Override
- public void updateCachedLocations(final TableName tableName, byte[] rowkey,
- final Object exception, final HRegionLocation source) {
- assert source != null;
- updateCachedLocations(tableName, source.getRegionInfo().getRegionName()
- , rowkey, exception, source.getServerName());
- }
-
/**
* Update the location with the new value (if the exception is a RegionMovedException)
* or delete it from the cache. Does nothing if we can be sure from the exception that
* the location is still accurate, or if the cache has already been updated.
* @param exception an object (to simplify user code) on which we will try to find a nested
- * or wrapped or both RegionMovedException
+ * or wrapped or both RegionMovedException
* @param source server that is the source of the location update.
*/
@Override
@@ -1916,84 +1820,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
metaCache.clearCache(regionInfo);
}
- @Override
- public void updateCachedLocations(final byte[] tableName, byte[] rowkey,
- final Object exception, final HRegionLocation source) {
- updateCachedLocations(TableName.valueOf(tableName), rowkey, exception, source);
- }
-
- /**
- * @deprecated since 0.96 Use {@link org.apache.hadoop.hbase.client.HTableInterface#batch} instead
- */
- @Override
- @Deprecated
- public void processBatch(List extends Row> list,
- final TableName tableName,
- ExecutorService pool,
- Object[] results) throws IOException, InterruptedException {
- // This belongs in HTable!!! Not in here. St.Ack
-
- // results must be the same size as list
- if (results.length != list.size()) {
- throw new IllegalArgumentException(
- "argument results must be the same size as argument list");
- }
- processBatchCallback(list, tableName, pool, results, null);
- }
-
- /**
- * @deprecated Unsupported API
- */
- @Override
- @Deprecated
- public void processBatch(List extends Row> list,
- final byte[] tableName,
- ExecutorService pool,
- Object[] results) throws IOException, InterruptedException {
- processBatch(list, TableName.valueOf(tableName), pool, results);
- }
-
- /**
- * Send the queries in parallel on the different region servers. Retries on failures.
- * If the method returns it means that there is no error, and the 'results' array will
- * contain no exception. On error, an exception is thrown, and the 'results' array will
- * contain results and exceptions.
- * @deprecated since 0.96
- * Use {@link org.apache.hadoop.hbase.client.HTable#processBatchCallback} instead
- */
- @Override
- @Deprecated
- public void processBatchCallback(
- List extends Row> list,
- TableName tableName,
- ExecutorService pool,
- Object[] results,
- Batch.Callback callback)
- throws IOException, InterruptedException {
-
- AsyncProcess.AsyncRequestFuture ars = this.asyncProcess.submitAll(
- pool, tableName, list, callback, results);
- ars.waitUntilDone();
- if (ars.hasError()) {
- throw ars.getErrors();
- }
- }
-
- /**
- * @deprecated Unsupported API
- */
- @Override
- @Deprecated
- public void processBatchCallback(
- List extends Row> list,
- byte[] tableName,
- ExecutorService pool,
- Object[] results,
- Batch.Callback callback)
- throws IOException, InterruptedException {
- processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback);
- }
-
// For tests to override.
protected AsyncProcess createAsyncProcess(Configuration conf) {
// No default pool available.
@@ -2024,41 +1850,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return metaCache.getNumberOfCachedRegionLocations(tableName);
}
- /**
- * @deprecated always return false since 0.99
- */
- @Override
- @Deprecated
- public void setRegionCachePrefetch(final TableName tableName, final boolean enable) {
- }
-
- /**
- * @deprecated always return false since 0.99
- */
- @Override
- @Deprecated
- public void setRegionCachePrefetch(final byte[] tableName,
- final boolean enable) {
- }
-
- /**
- * @deprecated always return false since 0.99
- */
- @Override
- @Deprecated
- public boolean getRegionCachePrefetch(TableName tableName) {
- return false;
- }
-
- /**
- * @deprecated always return false since 0.99
- */
- @Override
- @Deprecated
- public boolean getRegionCachePrefetch(byte[] tableName) {
- return false;
- }
-
@Override
public void abort(final String msg, Throwable t) {
if (t instanceof KeeperException.SessionExpiredException
@@ -2133,146 +1924,20 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
close();
}
- /**
- * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#listTables()} instead
- */
- @Deprecated
- @Override
- public HTableDescriptor[] listTables() throws IOException {
- MasterKeepAliveConnection master = getKeepAliveMasterService();
- try {
- MasterProtos.GetTableDescriptorsRequest req =
- RequestConverter.buildGetTableDescriptorsRequest((List)null);
- return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
- } catch (ServiceException se) {
- throw ProtobufUtil.getRemoteException(se);
- } finally {
- master.close();
- }
- }
-
- /**
- * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#listTableNames()} instead
- */
- @Deprecated
- @Override
- public String[] getTableNames() throws IOException {
- TableName[] tableNames = listTableNames();
- String[] result = new String[tableNames.length];
- for (int i = 0; i < tableNames.length; i++) {
- result[i] = tableNames[i].getNameAsString();
- }
- return result;
- }
-
- /**
- * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#listTableNames()} instead
- */
- @Deprecated
- @Override
- public TableName[] listTableNames() throws IOException {
- MasterKeepAliveConnection master = getKeepAliveMasterService();
- try {
- return ProtobufUtil.getTableNameArray(master.getTableNames(null,
- MasterProtos.GetTableNamesRequest.newBuilder().build())
- .getTableNamesList());
- } catch (ServiceException se) {
- throw ProtobufUtil.getRemoteException(se);
- } finally {
- master.close();
- }
- }
-
- /**
- * @deprecated Use {@link
- * org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)} instead
- */
- @Deprecated
- @Override
- public HTableDescriptor[] getHTableDescriptorsByTableName(
- List tableNames) throws IOException {
- if (tableNames == null || tableNames.isEmpty()) return new HTableDescriptor[0];
- MasterKeepAliveConnection master = getKeepAliveMasterService();
- try {
- MasterProtos.GetTableDescriptorsRequest req =
- RequestConverter.buildGetTableDescriptorsRequest(tableNames);
- return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
- } catch (ServiceException se) {
- throw ProtobufUtil.getRemoteException(se);
- } finally {
- master.close();
- }
- }
-
- /**
- * @deprecated Use
- * {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)}
- * instead
- */
- @Deprecated
- @Override
- public HTableDescriptor[] getHTableDescriptors(List names) throws IOException {
- List tableNames = new ArrayList(names.size());
- for(String name : names) {
- tableNames.add(TableName.valueOf(name));
- }
-
- return getHTableDescriptorsByTableName(tableNames);
- }
-
@Override
public NonceGenerator getNonceGenerator() {
return nonceGenerator;
}
- /**
- * Connects to the master to get the table descriptor.
- * @param tableName table name
- * @throws java.io.IOException if the connection to master fails or if the table
- * is not found.
- * @deprecated Use {@link
- * org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)}
- * instead
- */
- @Deprecated
- @Override
- public HTableDescriptor getHTableDescriptor(final TableName tableName)
- throws IOException {
- if (tableName == null) return null;
- MasterKeepAliveConnection master = getKeepAliveMasterService();
- MasterProtos.GetTableDescriptorsResponse htds;
- try {
- MasterProtos.GetTableDescriptorsRequest req =
- RequestConverter.buildGetTableDescriptorsRequest(tableName);
- htds = master.getTableDescriptors(null, req);
- } catch (ServiceException se) {
- throw ProtobufUtil.getRemoteException(se);
- } finally {
- master.close();
- }
- if (!htds.getTableSchemaList().isEmpty()) {
- return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
- }
- throw new TableNotFoundException(tableName.getNameAsString());
- }
-
- /**
- * @deprecated Use {@link
- * org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)}
- * instead
- */
- @Deprecated
- @Override
- public HTableDescriptor getHTableDescriptor(final byte[] tableName)
- throws IOException {
- return getHTableDescriptor(TableName.valueOf(tableName));
- }
-
@Override
public TableState getTableState(TableName tableName) throws IOException {
- if (this.closed) throw new IOException(toString() + " closed");
+ if (this.closed) {
+ throw new IOException(toString() + " closed");
+ }
TableState tableState = MetaTableAccessor.getTableState(this, tableName);
- if (tableState == null) throw new TableNotFoundException(tableName);
+ if (tableState == null) {
+ throw new TableNotFoundException(tableName);
+ }
return tableState;
}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index 363a0e0..df89622 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -17,9 +17,11 @@
*/
package org.apache.hadoop.hbase.client;
+import com.google.common.annotations.VisibleForTesting;
+
import java.io.IOException;
-import java.util.Random;
import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
@@ -32,8 +34,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
-import com.google.common.annotations.VisibleForTesting;
-
/**
* Utility used by client connections.
*/
@@ -42,12 +42,11 @@ public final class ConnectionUtils {
private ConnectionUtils() {}
- private static final Random RANDOM = new Random();
/**
* Calculate pause time.
* Built on {@link HConstants#RETRY_BACKOFF}.
- * @param pause
- * @param tries
+ * @param pause time to pause
+ * @param tries amount of tries
* @return How long to wait after tries retries
*/
public static long getPauseTime(final long pause, final int tries) {
@@ -60,18 +59,19 @@ public final class ConnectionUtils {
}
long normalPause = pause * HConstants.RETRY_BACKOFF[ntries];
- long jitter = (long)(normalPause * RANDOM.nextFloat() * 0.01f); // 1% possible jitter
+ // 1% possible jitter
+ long jitter = (long) (normalPause * ThreadLocalRandom.current().nextFloat() * 0.01f);
return normalPause + jitter;
}
/**
- * Adds / subs a 10% jitter to a pause time. Minimum is 1.
+ * Adds / subs an up to 50% jitter to a pause time. Minimum is 1.
* @param pause the expected pause.
* @param jitter the jitter ratio, between 0 and 1, exclusive.
*/
public static long addJitter(final long pause, final float jitter) {
- float lag = pause * (RANDOM.nextFloat() - 0.5f) * jitter;
+ float lag = pause * (ThreadLocalRandom.current().nextFloat() - 0.5f) * jitter;
long newPause = pause + (long) lag;
if (newPause <= 0) {
return 1;
@@ -90,7 +90,7 @@ public final class ConnectionUtils {
}
/**
- * Changes the configuration to set the number of retries needed when using HConnection
+ * Changes the configuration to set the number of retries needed when using Connection
* internally, e.g. for updating catalog tables, etc.
* Call this method before we create any Connections.
* @param c The Configuration instance to set the retries into.
@@ -106,7 +106,7 @@ public final class ConnectionUtils {
int serversideMultiplier = c.getInt("hbase.client.serverside.retries.multiplier", 10);
int retries = hcRetries * serversideMultiplier;
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
- log.info(sn + " server-side HConnection retries=" + retries);
+ log.info(sn + " server-side Connection retries=" + retries);
}
/**
@@ -119,7 +119,7 @@ public final class ConnectionUtils {
* @param admin the admin interface of the local server
* @param client the client interface of the local server
* @return an short-circuit connection.
- * @throws IOException
+ * @throws IOException if IO failure occurred
*/
public static ClusterConnection createShortCircuitConnection(final Configuration conf,
ExecutorService pool, User user, final ServerName serverName,
@@ -130,9 +130,8 @@ public final class ConnectionUtils {
}
return new ConnectionImplementation(conf, pool, user) {
@Override
- public AdminService.BlockingInterface getAdmin(ServerName sn, boolean getMaster)
- throws IOException {
- return serverName.equals(sn) ? admin : super.getAdmin(sn, getMaster);
+ public AdminService.BlockingInterface getAdmin(ServerName sn) throws IOException {
+ return serverName.equals(sn) ? admin : super.getAdmin(sn);
}
@Override
@@ -148,7 +147,7 @@ public final class ConnectionUtils {
*/
@VisibleForTesting
public static void setupMasterlessConnection(Configuration conf) {
- conf.set(HConnection.HBASE_CLIENT_CONNECTION_IMPL,
+ conf.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL,
MasterlessConnection.class.getName());
}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index a2ef34b..e21a5d2 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -18,6 +18,10 @@
*/
package org.apache.hadoop.hbase.client;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
+
import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
@@ -176,10 +180,6 @@ import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ServiceException;
-
/**
* HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
* this is an HBase-internal class as defined in
@@ -313,9 +313,9 @@ public class HBaseAdmin implements Admin {
}
}
- /** @return HConnection used by this object. */
+ /** @return Connection used by this object. */
@Override
- public HConnection getConnection() {
+ public Connection getConnection() {
return connection;
}
@@ -404,11 +404,11 @@ public class HBaseAdmin implements Admin {
@Override
public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
- return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
+ return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
operationTimeout, rpcTimeout);
}
- static HTableDescriptor getTableDescriptor(final TableName tableName, HConnection connection,
+ static HTableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
int operationTimeout, int rpcTimeout) throws IOException {
if (tableName == null) return null;
@@ -588,7 +588,7 @@ public class HBaseAdmin implements Admin {
protected Void postOperationResult(final Void result, final long deadlineTs)
throws IOException, TimeoutException {
// Delete cached information to prevent clients from using old locations
- getAdmin().getConnection().clearRegionCache(getTableName());
+ ((ClusterConnection) getAdmin().getConnection()).clearRegionCache(getTableName());
return super.postOperationResult(result, deadlineTs);
}
}
@@ -843,7 +843,7 @@ public class HBaseAdmin implements Admin {
@Override
public boolean isTableAvailable(TableName tableName) throws IOException {
- return connection.isTableAvailable(tableName);
+ return connection.isTableAvailable(tableName, null);
}
@Override
@@ -1701,7 +1701,7 @@ public class HBaseAdmin implements Admin {
* @param regionName Name of a region.
* @return a pair of HRegionInfo and ServerName if regionName is
* a verified region name (we call {@link
- * MetaTableAccessor#getRegionLocation(HConnection, byte[])}
+ * MetaTableAccessor#getRegionLocation(Connection, byte[])}
* else null.
* Throw IllegalArgumentException if regionName is null.
* @throws IOException
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
deleted file mode 100644
index cc5e9fa..0000000
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
+++ /dev/null
@@ -1,626 +0,0 @@
-/**
- *
- * 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.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.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
-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;
-
-/**
- * A cluster connection. Knows how to find the master, locate regions out on the cluster,
- * keeps a cache of locations and then knows how to re-calibrate after they move. You need one
- * of these to talk to your HBase cluster. {@link ConnectionFactory} manages instances of this
- * class. See it for how to get one of these.
- *
- * This is NOT a connection to a particular server but to ALL servers in the cluster. Individual
- * connections are managed at a lower level.
- *
- *
HConnections are used by {@link HTable} mostly but also by
- * {@link HBaseAdmin}, and {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}.
- *
- * @see ConnectionFactory
- * @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-@Deprecated
-public interface HConnection extends Connection {
- /**
- * Key for configuration in Configuration whose value is the class we implement making a
- * new HConnection instance.
- */
- public static final String HBASE_CLIENT_CONNECTION_IMPL = "hbase.client.connection.impl";
-
- /**
- * @return Configuration instance being used by this HConnection instance.
- */
- @Override
- Configuration getConfiguration();
-
- /**
- * Retrieve an HTableInterface implementation for access to a table.
- * The returned HTableInterface is not thread safe, a new instance should
- * be created for each using thread.
- * This is a lightweight operation, pooling or caching of the returned HTableInterface
- * is neither required nor desired.
- * (created with {@link ConnectionFactory#createConnection(Configuration)}).
- * @param tableName
- * @return an HTable to use for interactions with this table
- */
- public HTableInterface getTable(String tableName) throws IOException;
-
- /**
- * Retrieve an HTableInterface implementation for access to a table.
- * The returned HTableInterface is not thread safe, a new instance should
- * be created for each using thread.
- * This is a lightweight operation, pooling or caching of the returned HTableInterface
- * is neither required nor desired.
- * (created with {@link ConnectionFactory#createConnection(Configuration)}).
- * @param tableName
- * @return an HTable to use for interactions with this table
- */
- public HTableInterface getTable(byte[] tableName) throws IOException;
-
- /**
- * Retrieve an HTableInterface implementation for access to a table.
- * The returned HTableInterface is not thread safe, a new instance should
- * be created for each using thread.
- * This is a lightweight operation, pooling or caching of the returned HTableInterface
- * is neither required nor desired.
- * (created with {@link ConnectionFactory#createConnection(Configuration)}).
- * @param tableName
- * @return an HTable to use for interactions with this table
- */
- @Override
- public HTableInterface getTable(TableName tableName) throws IOException;
-
- /**
- * Retrieve an HTableInterface implementation for access to a table.
- * The returned HTableInterface is not thread safe, a new instance should
- * be created for each using thread.
- * This is a lightweight operation, pooling or caching of the returned HTableInterface
- * is neither required nor desired.
- * (created with {@link ConnectionFactory#createConnection(Configuration)}).
- * @param tableName
- * @param pool The thread pool to use for batch operations, null to use a default pool.
- * @return an HTable to use for interactions with this table
- */
- public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException;
-
- /**
- * Retrieve an HTableInterface implementation for access to a table.
- * The returned HTableInterface is not thread safe, a new instance should
- * be created for each using thread.
- * This is a lightweight operation, pooling or caching of the returned HTableInterface
- * is neither required nor desired.
- * (created with {@link ConnectionFactory#createConnection(Configuration)}).
- * @param tableName
- * @param pool The thread pool to use for batch operations, null to use a default pool.
- * @return an HTable to use for interactions with this table
- */
- public HTableInterface getTable(byte[] tableName, ExecutorService pool) throws IOException;
-
- /**
- * Retrieve an HTableInterface implementation for access to a table.
- * The returned HTableInterface is not thread safe, a new instance should
- * be created for each using thread.
- * This is a lightweight operation, pooling or caching of the returned HTableInterface
- * is neither required nor desired.
- * (created with {@link ConnectionFactory#createConnection(Configuration)}).
- * @param tableName table to get interface for
- * @param pool The thread pool to use for batch operations, null to use a default pool.
- * @return an HTable to use for interactions with this table
- */
- @Override
- public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException;
-
- /**
- * Retrieve a RegionLocator implementation to inspect region information on a table. The returned
- * RegionLocator is not thread-safe, so a new instance should be created for each using thread.
- *
- * This is a lightweight operation. Pooling or caching of the returned RegionLocator is neither
- * required nor desired.
- * @param tableName Name of the table who's region is to be examined
- * @return A RegionLocator instance
- */
- @Override
- public RegionLocator getRegionLocator(TableName tableName) throws IOException;
-
- /**
- * Retrieve an Admin implementation to administer an HBase cluster.
- * The returned Admin is not guaranteed to be thread-safe. A new instance should be created for
- * each using thread. This is a lightweight operation. Pooling or caching of the returned
- * Admin is not recommended.
- *
- * @return an Admin instance for cluster administration
- */
- @Override
- Admin getAdmin() throws IOException;
-
- /** @return - true if the master server is running
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- boolean isMasterRunning()
- throws MasterNotRunningException, ZooKeeperConnectionException;
-
- /**
- * A table that isTableEnabled == false and isTableDisabled == false
- * is possible. This happens when a table has a lot of regions
- * that must be processed.
- * @param tableName table name
- * @return true if the table is enabled, false otherwise
- * @throws IOException if a remote or network exception occurs
- */
- boolean isTableEnabled(TableName tableName) throws IOException;
-
- /**
- * @deprecated instead use {@link #isTableEnabled(TableName)}
- */
- @Deprecated
- boolean isTableEnabled(byte[] tableName) throws IOException;
-
- /**
- * @param tableName table name
- * @return true if the table is disabled, false otherwise
- * @throws IOException if a remote or network exception occurs
- */
- boolean isTableDisabled(TableName tableName) throws IOException;
-
- /**
- * @deprecated instead use {@link #isTableDisabled(TableName)}
- */
- @Deprecated
- boolean isTableDisabled(byte[] tableName) throws IOException;
-
- /**
- * Retrieve TableState, represent current table state.
- * @param tableName table state for
- * @return state of the table
- */
- public TableState getTableState(TableName tableName) throws IOException;
-
- /**
- * @param tableName table name
- * @return true if all regions of the table are available, false otherwise
- * @throws IOException if a remote or network exception occurs
- */
- boolean isTableAvailable(TableName tableName) throws IOException;
-
- /**
- * @deprecated instead use {@link #isTableAvailable(TableName)}
- */
- @Deprecated
- boolean isTableAvailable(byte[] tableName) throws IOException;
-
- /**
- * Use this api to check if the table has been created with the specified number of
- * splitkeys which was used while creating the given table.
- * Note : If this api is used after a table's region gets splitted, the api may return
- * false.
- * @param tableName tableName
- * @param splitKeys splitKeys used while creating table
- * @throws IOException if a remote or network exception occurs
- * @deprecated internal method, do not use through HConnection */
- @Deprecated
- boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException;
-
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException;
-
- /**
- * List all the userspace tables. In other words, scan the hbase:meta table.
- *
- * @return - returns an array of HTableDescriptors
- * @throws IOException if a remote or network exception occurs
- * @deprecated Use {@link Admin#listTables()} instead.
- */
- @Deprecated
- HTableDescriptor[] listTables() throws IOException;
-
- // This is a bit ugly - We call this getTableNames in 0.94 and the
- // successor function, returning TableName, listTableNames in later versions
- // because Java polymorphism doesn't consider return value types
-
- /**
- * @deprecated Use {@link Admin#listTableNames()} instead.
- */
- @Deprecated
- String[] getTableNames() throws IOException;
-
- /**
- * @deprecated Use {@link Admin#listTables()} instead.
- */
- @Deprecated
- TableName[] listTableNames() throws IOException;
-
- /**
- * @param tableName table name
- * @return table metadata
- * @throws IOException if a remote or network exception occurs
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- HTableDescriptor getHTableDescriptor(TableName tableName)
- throws IOException;
-
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- HTableDescriptor getHTableDescriptor(byte[] tableName)
- throws IOException;
-
- /**
- * Find the location of the region of tableName that row
- * lives in.
- * @param tableName name of the table row is in
- * @param row row key you're trying to find the region of
- * @return HRegionLocation that describes where to find the region in
- * question
- * @throws IOException if a remote or network exception occurs
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- public HRegionLocation locateRegion(final TableName tableName,
- final byte [] row) throws IOException;
-
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- public HRegionLocation locateRegion(final byte[] tableName,
- final byte [] row) throws IOException;
-
- /**
- * Allows flushing the region cache.
- * @deprecated internal method, do not use through HConnection */
- @Deprecated
- void clearRegionCache();
-
- /**
- * Allows flushing the region cache of all locations that pertain to
- * tableName
- * @param tableName Name of the table whose regions we are to remove from
- * cache.
- * @deprecated internal method, do not use through HConnection */
- @Deprecated
- void clearRegionCache(final TableName tableName);
-
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- void clearRegionCache(final byte[] tableName);
-
- /**
- * Deletes cached locations for the specific region.
- * @param location The location object for the region, to be purged from cache.
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- void deleteCachedRegionLocation(final HRegionLocation location);
-
- /**
- * Find the location of the region of tableName that row
- * lives in, ignoring any value that might be in the cache.
- * @param tableName name of the table row is in
- * @param row row key you're trying to find the region of
- * @return HRegionLocation that describes where to find the region in
- * question
- * @throws IOException if a remote or network exception occurs
- * @deprecated internal method, do not use through HConnection */
- @Deprecated
- HRegionLocation relocateRegion(final TableName tableName,
- final byte [] row) throws IOException;
-
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- HRegionLocation relocateRegion(final byte[] tableName,
- final byte [] row) throws IOException;
-
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- void updateCachedLocations(TableName tableName, byte[] rowkey,
- Object exception, HRegionLocation source);
-
- /**
- * Update the location cache. This is used internally by HBase, in most cases it should not be
- * used by the client application.
- * @param tableName the table name
- * @param regionName the regionName
- * @param rowkey the row
- * @param exception the exception if any. Can be null.
- * @param source the previous location
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- void updateCachedLocations(TableName tableName, byte[] regionName, byte[] rowkey,
- Object exception, ServerName source);
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- void updateCachedLocations(byte[] tableName, byte[] rowkey,
- Object exception, HRegionLocation source);
-
- /**
- * Gets the location of the region of regionName.
- * @param regionName name of the region to locate
- * @return HRegionLocation that describes where to find the region in
- * question
- * @throws IOException if a remote or network exception occurs
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- HRegionLocation locateRegion(final byte[] regionName)
- throws IOException;
-
- /**
- * Gets the locations of all regions in the specified table, tableName.
- * @param tableName table to get regions of
- * @return list of region locations for all regions of table
- * @throws IOException
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- List locateRegions(final TableName tableName) throws IOException;
-
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- List locateRegions(final byte[] tableName) throws IOException;
-
- /**
- * Gets the locations of all regions in the specified table, tableName.
- * @param tableName table to get regions of
- * @param useCache Should we use the cache to retrieve the region information.
- * @param offlined True if we are to include offlined regions, false and we'll leave out offlined
- * regions from returned list.
- * @return list of region locations for all regions of table
- * @throws IOException
- * @deprecated internal method, do not use thru HConnection
- */
- @Deprecated
- public List locateRegions(final TableName tableName,
- final boolean useCache,
- final boolean offlined) throws IOException;
-
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- public List locateRegions(final byte[] tableName,
- final boolean useCache,
- final boolean offlined) throws IOException;
-
- /**
- * Returns a {@link MasterKeepAliveConnection} to the active master
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- MasterService.BlockingInterface getMaster() throws IOException;
-
-
- /**
- * Establishes a connection to the region server at the specified address.
- * @param serverName
- * @return proxy for HRegionServer
- * @throws IOException if a remote or network exception occurs
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
-
- /**
- * Establishes a connection to the region server at the specified address, and returns
- * a region client protocol.
- *
- * @param serverName
- * @return ClientProtocol proxy for RegionServer
- * @throws IOException if a remote or network exception occurs
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
-
- /**
- * Establishes a connection to the region server at the specified address.
- * @param serverName
- * @param getMaster do we check if master is alive
- * @return proxy for HRegionServer
- * @throws IOException if a remote or network exception occurs
- * @deprecated You can pass master flag but nothing special is done.
- */
- @Deprecated
- AdminService.BlockingInterface getAdmin(final ServerName serverName, boolean getMaster)
- throws IOException;
-
- /**
- * Find region location hosting passed row
- * @param tableName table name
- * @param row Row to find.
- * @param reload If true do not use cache, otherwise bypass.
- * @return Location of row.
- * @throws IOException if a remote or network exception occurs
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- HRegionLocation getRegionLocation(TableName tableName, byte [] row,
- boolean reload)
- throws IOException;
-
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- HRegionLocation getRegionLocation(byte[] tableName, byte [] row,
- boolean reload)
- throws IOException;
-
- /**
- * Process a mixed batch of Get, Put and Delete actions. All actions for a
- * RegionServer are forwarded in one RPC call.
- *
- *
- * @param actions The collection of actions.
- * @param tableName Name of the hbase table
- * @param pool thread pool for parallel execution
- * @param results An empty array, same size as list. If an exception is thrown,
- * you can test here for partial results, and to determine which actions
- * processed successfully.
- * @throws IOException if there are problems talking to META. Per-item
- * exceptions are stored in the results array.
- * @deprecated since 0.96 - Use {@link HTableInterface#batch} instead
- */
- @Deprecated
- void processBatch(List extends Row> actions, final TableName tableName,
- ExecutorService pool, Object[] results) throws IOException, InterruptedException;
-
- /**
- * @deprecated internal method, do not use through HConnection
- */
- @Deprecated
- void processBatch(List extends Row> actions, final byte[] tableName,
- ExecutorService pool, Object[] results) throws IOException, InterruptedException;
-
- /**
- * Parameterized batch processing, allowing varying return types for different
- * {@link Row} implementations.
- * @deprecated since 0.96 - Use {@link HTableInterface#batchCallback} instead
- */
- @Deprecated
- public void processBatchCallback(List extends Row> list,
- final TableName tableName,
- ExecutorService pool,
- Object[] results,
- Batch.Callback callback) throws IOException, InterruptedException;
-
- /**
- * @deprecated Unsupported API
- */
- @Deprecated
- public void processBatchCallback(List extends Row> list,
- final byte[] tableName,
- ExecutorService pool,
- Object[] results,
- Batch.Callback callback) throws IOException, InterruptedException;
-
- /**
- * @deprecated does nothing since since 0.99
- **/
- @Deprecated
- public void setRegionCachePrefetch(final TableName tableName,
- final boolean enable);
-
- /**
- * @deprecated does nothing since 0.99
- **/
- @Deprecated
- public void setRegionCachePrefetch(final byte[] tableName,
- final boolean enable);
-
- /**
- * @deprecated always return false since 0.99
- **/
- @Deprecated
- boolean getRegionCachePrefetch(final TableName tableName);
-
- /**
- * @deprecated always return false since 0.99
- **/
- @Deprecated
- boolean getRegionCachePrefetch(final byte[] tableName);
-
- /**
- * @return the number of region servers that are currently running
- * @throws IOException if a remote or network exception occurs
- * @deprecated This method will be changed from public to package protected.
- */
- @Deprecated
- int getCurrentNrHRS() throws IOException;
-
- /**
- * @param tableNames List of table names
- * @return HTD[] table metadata
- * @throws IOException if a remote or network exception occurs
- * @deprecated Use {@link Admin#getTableDescriptor(TableName)} instead.
- */
- @Deprecated
- HTableDescriptor[] getHTableDescriptorsByTableName(List tableNames) throws IOException;
-
- /**
- * @deprecated since 0.96.0
- */
- @Deprecated
- HTableDescriptor[] getHTableDescriptors(List tableNames) throws
- IOException;
-
- /**
- * @return true if this connection is closed
- */
- @Override
- boolean isClosed();
-
-
- /**
- * Clear any caches that pertain to server name sn.
- * @param sn A server name
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- void clearCaches(final ServerName sn);
-
- /**
- * This function allows HBaseAdmin and potentially others to get a shared MasterService
- * connection.
- * @return The shared instance. Never returns null.
- * @throws MasterNotRunningException
- * @deprecated Since 0.96.0
- */
- // TODO: Why is this in the public interface when the returned type is shutdown package access?
- @Deprecated
- MasterKeepAliveConnection getKeepAliveMasterService()
- throws MasterNotRunningException;
-
- /**
- * @param serverName
- * @return true if the server is known as dead, false otherwise.
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- boolean isDeadServer(ServerName serverName);
-
- /**
- * @return Nonce generator for this HConnection; may be null if disabled in configuration.
- * @deprecated internal method, do not use thru HConnection */
- @Deprecated
- public NonceGenerator getNonceGenerator();
-}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index bf9ec22..54fbfe9 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -18,6 +18,12 @@
*/
package org.apache.hadoop.hbase.client;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
@@ -68,12 +74,6 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
/**
* An implementation of {@link Table}. Used to communicate with a single HBase table.
* Lightweight. Get as needed and just close when done.
@@ -100,14 +100,13 @@ import com.google.protobuf.ServiceException;
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
-public class HTable implements HTableInterface {
+public class HTable implements Table {
private static final Log LOG = LogFactory.getLog(HTable.class);
protected ClusterConnection connection;
private final TableName tableName;
private volatile Configuration configuration;
private ConnectionConfiguration connConfiguration;
protected BufferedMutatorImpl mutator;
- private boolean autoFlush = true;
private boolean closed = false;
protected int scannerCaching;
protected long scannerMaxResultSize;
@@ -149,7 +148,7 @@ public class HTable implements HTableInterface {
* Used by HBase internally. DO NOT USE. See {@link ConnectionFactory} class comment for how to
* get a {@link Table} instance (use {@link Table} instead of {@link HTable}).
* @param tableName Name of the table.
- * @param connection HConnection to be used.
+ * @param connection Connection to be used.
* @param pool ExecutorService to be used.
* @throws IOException if a remote or network exception occurs
*/
@@ -237,14 +236,6 @@ public class HTable implements HTableInterface {
return configuration;
}
- /**
- * {@inheritDoc}
- */
- @Override
- public byte [] getTableName() {
- return this.tableName.getName();
- }
-
@Override
public TableName getName() {
return tableName;
@@ -253,13 +244,10 @@ public class HTable implements HTableInterface {
/**
* INTERNAL Used by unit tests and tools to do low-level
* manipulations.
- * @return An HConnection instance.
- * @deprecated This method will be changed from public to package protected.
+ * @return A Connection instance.
*/
- // TODO(tsuna): Remove this. Unit tests shouldn't require public helpers.
- @Deprecated
@VisibleForTesting
- public HConnection getConnection() {
+ protected Connection getConnection() {
return this.connection;
}
@@ -331,7 +319,7 @@ public class HTable implements HTableInterface {
/**
* The underlying {@link HTable} must not be closed.
- * {@link HTableInterface#getScanner(Scan)} has other usage details.
+ * {@link Table#getScanner(Scan)} has other usage details.
*/
@Override
public ResultScanner getScanner(final Scan scan) throws IOException {
@@ -382,7 +370,7 @@ public class HTable implements HTableInterface {
/**
* The underlying {@link HTable} must not be closed.
- * {@link HTableInterface#getScanner(byte[])} has other usage details.
+ * {@link Table#getScanner(byte[])} has other usage details.
*/
@Override
public ResultScanner getScanner(byte [] family) throws IOException {
@@ -393,7 +381,7 @@ public class HTable implements HTableInterface {
/**
* The underlying {@link HTable} must not be closed.
- * {@link HTableInterface#getScanner(byte[], byte[])} has other usage details.
+ * {@link Table#getScanner(byte[], byte[])} has other usage details.
*/
@Override
public ResultScanner getScanner(byte [] family, byte [] qualifier)
@@ -500,9 +488,20 @@ public class HTable implements HTableInterface {
*/
@Override
public void batchCallback(
- final List extends Row> actions, final Object[] results, final Batch.Callback callback)
- throws IOException, InterruptedException {
- connection.processBatchCallback(actions, tableName, pool, results, callback);
+ final List extends Row> actions, final Object[] results, final Batch.Callback callback)
+ throws IOException, InterruptedException {
+ doBatchWithCallback(actions, results, callback, connection, pool, tableName);
+ }
+
+ public static void doBatchWithCallback(List extends Row> actions, Object[] results,
+ Callback callback, ClusterConnection connection, ExecutorService pool, TableName tableName)
+ throws InterruptedIOException, RetriesExhaustedWithDetailsException {
+ AsyncRequestFuture ars = connection.getAsyncProcess().submitAll(
+ pool, tableName, actions, callback, results);
+ ars.waitUntilDone();
+ if (ars.hasError()) {
+ throw ars.getErrors();
+ }
}
/**
@@ -564,9 +563,7 @@ public class HTable implements HTableInterface {
@Override
public void put(final Put put) throws IOException {
getBufferedMutator().mutate(put);
- if (autoFlush) {
- flushCommits();
- }
+ flushCommits();
}
/**
@@ -576,9 +573,7 @@ public class HTable implements HTableInterface {
@Override
public void put(final List puts) throws IOException {
getBufferedMutator().mutate(puts);
- if (autoFlush) {
- flushCommits();
- }
+ flushCommits();
}
/**
@@ -976,8 +971,7 @@ public class HTable implements HTableInterface {
* {@inheritDoc}
* @throws IOException
*/
- @Override
- public void flushCommits() throws IOException {
+ void flushCommits() throws IOException {
if (mutator == null) {
// nothing to flush if there's no mutator; don't bother creating one.
return;
@@ -991,10 +985,10 @@ public class HTable implements HTableInterface {
*
* @param list The collection of actions.
* @param results An empty array, same size as list. If an exception is thrown,
- * you can test here for partial results, and to determine which actions
- * processed successfully.
+ * you can test here for partial results, and to determine which actions
+ * processed successfully.
* @throws IOException if there are problems talking to META. Per-item
- * exceptions are stored in the results array.
+ * exceptions are stored in the results array.
*/
public void processBatchCallback(
final List extends Row> list, final Object[] results, final Batch.Callback callback)
@@ -1062,30 +1056,6 @@ public class HTable implements HTableInterface {
}
/**
- * {@inheritDoc}
- */
- @Override
- public boolean isAutoFlush() {
- return autoFlush;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void setAutoFlushTo(boolean autoFlush) {
- this.autoFlush = autoFlush;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
- this.autoFlush = autoFlush;
- }
-
- /**
* Returns the maximum size in bytes of the write buffer for this HTable.
*
* The default value comes from the configuration parameter
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
index 4cd81e7..9d41218 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Used to communicate with a single HBase table.
- * Obtain an instance from an {@link HConnection}.
+ * Obtain an instance from a {@link Connection}.
*
* @since 0.21.0
* @deprecated use {@link org.apache.hadoop.hbase.client.Table} instead
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
index a636533..66d3c21 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
@@ -26,11 +26,11 @@ import java.io.IOException;
* @param return type
*/
abstract class MasterCallable implements RetryingCallable, Closeable {
- protected HConnection connection;
+ protected ClusterConnection connection;
protected MasterKeepAliveConnection master;
- public MasterCallable(final HConnection connection) {
- this.connection = connection;
+ public MasterCallable(final Connection connection) {
+ this.connection = (ClusterConnection) connection;
}
@Override
@@ -41,7 +41,9 @@ abstract class MasterCallable implements RetryingCallable, Closeable {
@Override
public void close() throws IOException {
// The above prepare could fail but this would still be called though masterAdmin is null
- if (this.master != null) this.master.close();
+ if (this.master != null) {
+ this.master.close();
+ }
}
@Override
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
index a2ca975..6b06875 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
@@ -43,9 +43,10 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* Used to perform Put operations for a single row.
*
- * To perform a Put, instantiate a Put object with the row to insert to and
- * for eachumn to be inserted, execute {@link #addColumn(byte[], byte[], byte[]) add} or
- * {@link #addColumn(byte[], byte[], long, byte[]) add} if setting the timestamp.
+ * To perform a Put, instantiate a Put object with the row to insert to, and
+ * for each column to be inserted, execute {@link #addColumn(byte[], byte[],
+ * byte[]) add} or {@link #addColumn(byte[], byte[], long, byte[]) add} if
+ * setting the timestamp.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
index 725bec0..54c93a0 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
@@ -121,9 +121,9 @@ public abstract class RegionAdminServiceCallable implements RetryingCallable<
}
/**
- * @return {@link HConnection} instance used by this Callable.
+ * @return {@link Connection} instance used by this Callable.
*/
- HConnection getConnection() {
+ Connection getConnection() {
return this.connection;
}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
index 443026f..bfdb216 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
@@ -20,11 +20,11 @@ package org.apache.hadoop.hbase.client;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
/**
* POJO representing region server load
*/
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
public class RegionLoadStats {
int memstoreLoad;
int heapOccupancy;
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
index 1b0f387..f2cec97 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
@@ -577,12 +577,12 @@ public interface Table extends Closeable {
* early and throw SocketTimeoutException.
* @param operationTimeout the total timeout of each operation in millisecond.
*/
- public void setOperationTimeout(int operationTimeout);
+ void setOperationTimeout(int operationTimeout);
/**
* Get timeout (millisecond) of each operation for in Table instance.
*/
- public int getOperationTimeout();
+ int getOperationTimeout();
/**
* Set timeout (millisecond) of each rpc request in operations of this Table instance, will
@@ -591,11 +591,11 @@ public interface Table extends Closeable {
* retries exhausted or operation timeout reached.
* @param rpcTimeout the timeout of each rpc request in millisecond.
*/
- public void setRpcTimeout(int rpcTimeout);
+ void setRpcTimeout(int rpcTimeout);
/**
* Get timeout (millisecond) of each rpc request in this Table instance.
*/
- public int getRpcTimeout();
+ int getRpcTimeout();
}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java
index 04d4b41..34f7b23 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java
@@ -28,9 +28,9 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/**
* We inherit the current ZooKeeperWatcher implementation to change the semantic
* of the close: the new close won't immediately close the connection but
- * will have a keep alive. See {@link HConnection}.
+ * will have a keep alive. See {@link ConnectionImplementation}.
* This allows to make it available with a consistent interface. The whole
- * ZooKeeperWatcher use in HConnection will be then changed to remove the
+ * ZooKeeperWatcher use in ConnectionImplementation will be then changed to remove the
* watcher part.
*
* This class is intended to be used internally by HBase classes; but not by
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index a2ad2e7..d062448 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -516,9 +516,9 @@ public class ReplicationAdmin implements Closeable {
if (repPeers == null || repPeers.size() <= 0) {
throw new IllegalArgumentException("Found no peer cluster for replication.");
}
-
+
final TableName onlyTableNameQualifier = TableName.valueOf(tableName.getQualifierAsString());
-
+
for (ReplicationPeer repPeer : repPeers) {
Map> tableCFMap = repPeer.getTableCFs();
// TODO Currently peer TableCFs will not include namespace so we need to check only for table
@@ -595,20 +595,11 @@ public class ReplicationAdmin implements Closeable {
admin = this.connection.getAdmin();
HTableDescriptor htd = admin.getTableDescriptor(tableName);
if (isTableRepEnabled(htd) ^ isRepEnabled) {
- boolean isOnlineSchemaUpdateEnabled =
- this.connection.getConfiguration()
- .getBoolean("hbase.online.schema.update.enable", true);
- if (!isOnlineSchemaUpdateEnabled) {
- admin.disableTable(tableName);
- }
for (HColumnDescriptor hcd : htd.getFamilies()) {
hcd.setScope(isRepEnabled ? HConstants.REPLICATION_SCOPE_GLOBAL
: HConstants.REPLICATION_SCOPE_LOCAL);
}
admin.modifyTable(tableName, htd);
- if (!isOnlineSchemaUpdateEnabled) {
- admin.enableTable(tableName);
- }
}
} finally {
if (admin != null) {
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
index 63fd0a3..d36b158 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
@@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.filter;
+import java.util.Locale;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -53,8 +54,8 @@ public class SubstringComparator extends ByteArrayComparable {
* @param substr the substring
*/
public SubstringComparator(String substr) {
- super(Bytes.toBytes(substr.toLowerCase()));
- this.substr = substr.toLowerCase();
+ super(Bytes.toBytes(substr.toLowerCase(Locale.ROOT)));
+ this.substr = substr.toLowerCase(Locale.ROOT);
}
@Override
@@ -64,7 +65,7 @@ public class SubstringComparator extends ByteArrayComparable {
@Override
public int compareTo(byte[] value, int offset, int length) {
- return Bytes.toString(value, offset, length).toLowerCase().contains(substr) ? 0
+ return Bytes.toString(value, offset, length).toLowerCase(Locale.ROOT).contains(substr) ? 0
: 1;
}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 71c8875..3d3339a 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -30,6 +30,7 @@ import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -266,7 +267,7 @@ public abstract class AbstractRpcClient implements RpcClient {
@Override
public BlockingRpcChannel createBlockingRpcChannel(final ServerName sn, final User ticket,
- int defaultOperationTimeout) {
+ int defaultOperationTimeout) throws UnknownHostException {
return new BlockingRpcChannelImplementation(this, sn, ticket, defaultOperationTimeout);
}
@@ -332,8 +333,12 @@ public abstract class AbstractRpcClient implements RpcClient {
* @param channelOperationTimeout - the default timeout when no timeout is given
*/
protected BlockingRpcChannelImplementation(final AbstractRpcClient rpcClient,
- final ServerName sn, final User ticket, int channelOperationTimeout) {
+ final ServerName sn, final User ticket, int channelOperationTimeout)
+ throws UnknownHostException {
this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort());
+ if (this.isa.isUnresolved()) {
+ throw new UnknownHostException(sn.getHostname());
+ }
this.rpcClient = rpcClient;
this.ticket = ticket;
this.channelOperationTimeout = channelOperationTimeout;
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannelImpl.java hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannelImpl.java
index 2b9000a..6b7dc5b 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannelImpl.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannelImpl.java
@@ -38,6 +38,7 @@ import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
+import java.util.Locale;
import java.util.List;
import java.util.Map;
import java.util.Random;
@@ -231,6 +232,26 @@ public class AsyncRpcChannelImpl implements AsyncRpcChannel {
}
}
+ private void startConnectionWithEncryption(Channel ch) {
+ // for rpc encryption, the order of ChannelInboundHandler should be:
+ // LengthFieldBasedFrameDecoder->SaslClientHandler->LengthFieldBasedFrameDecoder
+ // Don't skip the first 4 bytes for length in beforeUnwrapDecoder,
+ // SaslClientHandler will handler this
+ ch.pipeline().addFirst("beforeUnwrapDecoder",
+ new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 0));
+ ch.pipeline().addLast("afterUnwrapDecoder",
+ new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4));
+ ch.pipeline().addLast(new AsyncServerResponseHandler(this));
+ List callsToWrite;
+ synchronized (pendingCalls) {
+ connected = true;
+ callsToWrite = new ArrayList(pendingCalls.values());
+ }
+ for (AsyncCall call : callsToWrite) {
+ writeRequest(call);
+ }
+ }
+
/**
* Get SASL handler
* @param bootstrap to reconnect to
@@ -242,7 +263,8 @@ public class AsyncRpcChannelImpl implements AsyncRpcChannel {
return new SaslClientHandler(realTicket, authMethod, token, serverPrincipal,
client.fallbackAllowed,
client.conf.get("hbase.rpc.protection",
- SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase()),
+ SaslUtil.QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)),
+ getChannelHeaderBytes(authMethod),
new SaslClientHandler.SaslExceptionHandler() {
@Override
public void handle(int retryCount, Random random, Throwable cause) {
@@ -261,6 +283,11 @@ public class AsyncRpcChannelImpl implements AsyncRpcChannel {
public void onSuccess(Channel channel) {
startHBaseConnection(channel);
}
+
+ @Override
+ public void onSaslProtectionSucess(Channel channel) {
+ startConnectionWithEncryption(channel);
+ }
});
}
@@ -341,6 +368,25 @@ public class AsyncRpcChannelImpl implements AsyncRpcChannel {
* @throws java.io.IOException on failure to write
*/
private ChannelFuture writeChannelHeader(Channel channel) throws IOException {
+ RPCProtos.ConnectionHeader header = getChannelHeader(authMethod);
+ int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(header);
+ ByteBuf b = channel.alloc().directBuffer(totalSize);
+
+ b.writeInt(header.getSerializedSize());
+ b.writeBytes(header.toByteArray());
+
+ return channel.writeAndFlush(b);
+ }
+
+ private byte[] getChannelHeaderBytes(AuthMethod authMethod) {
+ RPCProtos.ConnectionHeader header = getChannelHeader(authMethod);
+ ByteBuffer b = ByteBuffer.allocate(header.getSerializedSize() + 4);
+ b.putInt(header.getSerializedSize());
+ b.put(header.toByteArray());
+ return b.array();
+ }
+
+ private RPCProtos.ConnectionHeader getChannelHeader(AuthMethod authMethod) {
RPCProtos.ConnectionHeader.Builder headerBuilder = RPCProtos.ConnectionHeader.newBuilder()
.setServiceName(serviceName);
@@ -357,16 +403,7 @@ public class AsyncRpcChannelImpl implements AsyncRpcChannel {
}
headerBuilder.setVersionInfo(ProtobufUtil.getVersionInfo());
- RPCProtos.ConnectionHeader header = headerBuilder.build();
-
- int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(header);
-
- ByteBuf b = channel.alloc().directBuffer(totalSize);
-
- b.writeInt(header.getSerializedSize());
- b.writeBytes(header.toByteArray());
-
- return channel.writeAndFlush(b);
+ return headerBuilder.build();
}
/**
@@ -443,7 +480,7 @@ public class AsyncRpcChannelImpl implements AsyncRpcChannel {
throw new IOException("Can't obtain server Kerberos config key from SecurityInfo");
}
this.serverPrincipal = SecurityUtil.getServerPrincipal(client.conf.get(serverKey),
- address.getAddress().getCanonicalHostName().toLowerCase());
+ address.getAddress().getCanonicalHostName().toLowerCase(Locale.ROOT));
if (LOG.isDebugEnabled()) {
LOG.debug("RPC Server Kerberos principal name for service=" + serviceName + " is "
+ serverPrincipal);
@@ -661,7 +698,7 @@ public class AsyncRpcChannelImpl implements AsyncRpcChannel {
} else {
String msg = "Couldn't setup connection for "
+ UserGroupInformation.getLoginUser().getUserName() + " to " + serverPrincipal;
- LOG.warn(msg);
+ LOG.warn(msg, ex);
throw (IOException) new IOException(msg).initCause(ex);
}
} else {
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
index 00eea7a..dc05af1 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
@@ -45,6 +45,7 @@ import java.security.PrivilegedExceptionAction;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
+import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
@@ -331,7 +332,7 @@ public class RpcClientImpl extends AbstractRpcClient {
"Can't obtain server Kerberos config key from SecurityInfo");
}
serverPrincipal = SecurityUtil.getServerPrincipal(
- conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase());
+ conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase(Locale.ROOT));
if (LOG.isDebugEnabled()) {
LOG.debug("RPC Server Kerberos principal name for service="
+ remoteId.getServiceName() + " is " + serverPrincipal);
@@ -618,7 +619,7 @@ public class RpcClientImpl extends AbstractRpcClient {
final OutputStream out2) throws IOException {
saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed,
conf.get("hbase.rpc.protection",
- QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
+ QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)));
return saslRpcClient.saslConnect(in2, out2);
}
@@ -672,7 +673,7 @@ public class RpcClientImpl extends AbstractRpcClient {
String msg = "Couldn't setup connection for " +
UserGroupInformation.getLoginUser().getUserName() +
" to " + serverPrincipal;
- LOG.warn(msg);
+ LOG.warn(msg, ex);
throw (IOException) new IOException(msg).initCause(ex);
}
} else {
@@ -875,7 +876,7 @@ public class RpcClientImpl extends AbstractRpcClient {
}
protected void tracedWriteRequest(Call call, int priority, Span span) throws IOException {
- try (TraceScope ignored = Trace.continueSpan(span)) {
+ try (TraceScope ignored = Trace.startSpan("RpcClientImpl.tracedWriteRequest", span)) {
writeRequest(call, priority, span);
}
}
@@ -1201,9 +1202,8 @@ public class RpcClientImpl extends AbstractRpcClient {
}
if (connsToClose != null) {
for (Connection conn : connsToClose) {
- if (conn.markClosed(new InterruptedIOException("RpcClient is closing"))) {
- conn.close();
- }
+ conn.markClosed(new InterruptedIOException("RpcClient is closing"));
+ conn.close();
}
}
// wait until all connections are closed
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java
index af8ddd4..347d8a1 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.protobuf.ResponseConverter;
* Base class which provides clients with an RPC connection to
* call coprocessor endpoint {@link com.google.protobuf.Service}s.
* Note that clients should not use this class directly, except through
- * {@link org.apache.hadoop.hbase.client.HTableInterface#coprocessorService(byte[])}.
+ * {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])}.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index bbc13ab..fecc3c2 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -31,10 +31,13 @@ import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
+import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.util.NavigableSet;
+import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
@@ -1108,6 +1111,16 @@ public final class ProtobufUtil {
return builder.build();
}
+ static void setTimeRange(final MutationProto.Builder builder, final TimeRange timeRange) {
+ if (!timeRange.isAllTime()) {
+ HBaseProtos.TimeRange.Builder timeRangeBuilder =
+ HBaseProtos.TimeRange.newBuilder();
+ timeRangeBuilder.setFrom(timeRange.getMin());
+ timeRangeBuilder.setTo(timeRange.getMax());
+ builder.setTimeRange(timeRangeBuilder.build());
+ }
+ }
+
/**
* Convert a client Increment to a protobuf Mutate.
*
@@ -1123,13 +1136,7 @@ public final class ProtobufUtil {
builder.setNonce(nonce);
}
TimeRange timeRange = increment.getTimeRange();
- if (!timeRange.isAllTime()) {
- HBaseProtos.TimeRange.Builder timeRangeBuilder =
- HBaseProtos.TimeRange.newBuilder();
- timeRangeBuilder.setFrom(timeRange.getMin());
- timeRangeBuilder.setTo(timeRange.getMax());
- builder.setTimeRange(timeRangeBuilder.build());
- }
+ setTimeRange(builder, timeRange);
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry> family: increment.getFamilyCellMap().entrySet()) {
@@ -1250,6 +1257,9 @@ public final class ProtobufUtil {
final MutationProto.Builder builder, long nonce) throws IOException {
getMutationBuilderAndSetCommonFields(type, mutation, builder);
builder.setAssociatedCellCount(mutation.size());
+ if (mutation instanceof Increment) {
+ setTimeRange(builder, ((Increment)mutation).getTimeRange());
+ }
if (nonce != HConstants.NO_NONCE) {
builder.setNonce(nonce);
}
@@ -3445,7 +3455,7 @@ public final class ProtobufUtil {
*/
public static HBaseProtos.SnapshotDescription.Type
createProtosSnapShotDescType(String snapshotDesc) {
- return HBaseProtos.SnapshotDescription.Type.valueOf(snapshotDesc.toUpperCase());
+ return HBaseProtos.SnapshotDescription.Type.valueOf(snapshotDesc.toUpperCase(Locale.ROOT));
}
/**
@@ -3528,12 +3538,11 @@ public final class ProtobufUtil {
backupMasters.add(ProtobufUtil.toServerName(sn));
}
- Map rit = null;
- rit = new HashMap(proto.getRegionsInTransitionList().size());
+ Set rit = null;
+ rit = new HashSet(proto.getRegionsInTransitionList().size());
for (RegionInTransition region : proto.getRegionsInTransitionList()) {
- String key = new String(region.getSpec().getValue().toByteArray());
RegionState value = RegionState.convert(region.getRegionState());
- rit.put(key, value);
+ rit.add(value);
}
String[] masterCoprocessors = null;
@@ -3577,11 +3586,11 @@ public final class ProtobufUtil {
}
if (status.getRegionsInTransition() != null) {
- for (Map.Entry rit : status.getRegionsInTransition().entrySet()) {
- ClusterStatusProtos.RegionState rs = rit.getValue().convert();
+ for (RegionState rit : status.getRegionsInTransition()) {
+ ClusterStatusProtos.RegionState rs = rit.convert();
RegionSpecifier.Builder spec =
RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME);
- spec.setValue(ByteStringer.wrap(Bytes.toBytes(rit.getKey())));
+ spec.setValue(ByteStringer.wrap(rit.getRegion().getRegionName()));
RegionInTransition pbRIT =
RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build();
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 91e77ca..e264a4d 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.replication;
+import org.apache.commons.lang.reflect.ConstructorUtils;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
@@ -30,9 +31,11 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@InterfaceAudience.Private
public class ReplicationFactory {
- public static ReplicationQueues getReplicationQueues(final ZooKeeperWatcher zk,
- Configuration conf, Abortable abortable) {
- return new ReplicationQueuesZKImpl(zk, conf, abortable);
+ public static ReplicationQueues getReplicationQueues(ReplicationQueuesArguments args)
+ throws Exception {
+ Class> classToBuild = args.getConf().getClass("hbase.region.replica." +
+ "replication.ReplicationQueuesType", ReplicationQueuesZKImpl.class);
+ return (ReplicationQueues) ConstructorUtils.invokeConstructor(classToBuild, args);
}
public static ReplicationQueuesClient getReplicationQueuesClient(final ZooKeeperWatcher zk,
@@ -44,7 +47,7 @@ public class ReplicationFactory {
Abortable abortable) {
return getReplicationPeers(zk, conf, null, abortable);
}
-
+
public static ReplicationPeers getReplicationPeers(final ZooKeeperWatcher zk, Configuration conf,
final ReplicationQueuesClient queuesClient, Abortable abortable) {
return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable);
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
index 7799de6..1d2066c 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
@@ -96,8 +96,10 @@ public class ReplicationPeerConfig {
@Override
public String toString() {
StringBuilder builder = new StringBuilder("clusterKey=").append(clusterKey).append(",");
- builder.append("replicationEndpointImpl=").append(replicationEndpointImpl).append(",")
- .append("tableCFs=").append(tableCFsMap.toString());
+ builder.append("replicationEndpointImpl=").append(replicationEndpointImpl).append(",");
+ if (tableCFsMap != null) {
+ builder.append("tableCFs=").append(tableCFsMap.toString());
+ }
return builder.toString();
}
}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index 09d2100..5af97c2 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -129,17 +129,6 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
- // Irrespective of bulk load hfile replication is enabled or not we add peerId node to
- // hfile-refs node -- HBASE-15397
- try {
- String peerId = ZKUtil.joinZNode(this.hfileRefsZNode, id);
- LOG.info("Adding peer " + peerId + " to hfile reference queue.");
- ZKUtil.createWithParents(this.zookeeper, peerId);
- } catch (KeeperException e) {
- throw new ReplicationException("Failed to add peer with id=" + id
- + ", node under hfile references node.", e);
- }
-
List listOfOps = new ArrayList();
ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(getPeerNode(id),
ReplicationSerDeHelper.toByteArray(peerConfig));
@@ -166,16 +155,6 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
+ " because that id does not exist.");
}
ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id));
- // Delete peerId node from hfile-refs node irrespective of whether bulk loaded hfile
- // replication is enabled or not
-
- String peerId = ZKUtil.joinZNode(this.hfileRefsZNode, id);
- try {
- LOG.info("Removing peer " + peerId + " from hfile reference queue.");
- ZKUtil.deleteNodeRecursively(this.zookeeper, peerId);
- } catch (NoNodeException e) {
- LOG.info("Did not find node " + peerId + " to delete.", e);
- }
} catch (KeeperException e) {
throw new ReplicationException("Could not remove peer with id=" + id, e);
}
@@ -550,6 +529,12 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
}
}
}
+ // Check for hfile-refs queue
+ if (-1 != ZKUtil.checkExists(zookeeper, hfileRefsZNode)
+ && queuesClient.getAllPeersFromHFileRefsQueue().contains(peerId)) {
+ throw new ReplicationException("Undeleted queue for peerId: " + peerId
+ + ", found in hfile-refs node path " + hfileRefsZNode);
+ }
} catch (KeeperException e) {
throw new ReplicationException("Could not check queues deleted with id=" + peerId, e);
}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
index 0d47a88..809b122 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
@@ -83,13 +83,13 @@ public interface ReplicationQueues {
/**
* Get a list of all WALs in the given queue.
* @param queueId a String that identifies the queue
- * @return a list of WALs, null if this region server is dead and has no outstanding queues
+ * @return a list of WALs, null if no such queue exists for this server
*/
List getLogsInQueue(String queueId);
/**
* Get a list of all queues for this region server.
- * @return a list of queueIds, null if this region server is dead and has no outstanding queues
+ * @return a list of queueIds, an empty list if this region server is dead and has no outstanding queues
*/
List getAllQueues();
@@ -110,10 +110,10 @@ public interface ReplicationQueues {
/**
* Checks if the provided znode is the same as this region server's
- * @param znode to check
+ * @param regionserver the id of the region server
* @return if this is this rs's znode
*/
- boolean isThisOurZnode(String znode);
+ boolean isThisOurRegionServer(String regionserver);
/**
* Add a peer to hfile reference queue if peer does not exist.
@@ -123,6 +123,12 @@ public interface ReplicationQueues {
void addPeerToHFileRefs(String peerId) throws ReplicationException;
/**
+ * Remove a peer from hfile reference queue.
+ * @param peerId peer cluster id to be removed
+ */
+ void removePeerFromHFileRefs(String peerId);
+
+ /**
* Add new hfile references to the queue.
* @param peerId peer cluster id to which the hfiles need to be replicated
* @param files list of hfile references to be added
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
new file mode 100644
index 0000000..4907b73
--- /dev/null
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
@@ -0,0 +1,66 @@
+/*
+ *
+ * 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.replication;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+
+@InterfaceAudience.Private
+public class ReplicationQueuesArguments {
+
+ private ZooKeeperWatcher zk;
+ private Configuration conf;
+ private Abortable abort;
+
+ public ReplicationQueuesArguments(Configuration conf, Abortable abort) {
+ this.conf = conf;
+ this.abort = abort;
+ }
+
+ public ReplicationQueuesArguments(Configuration conf, Abortable abort, ZooKeeperWatcher zk) {
+ this(conf, abort);
+ setZk(zk);
+ }
+
+ public ZooKeeperWatcher getZk() {
+ return zk;
+ }
+
+ public void setZk(ZooKeeperWatcher zk) {
+ this.zk = zk;
+ }
+
+ public Configuration getConf() {
+ return conf;
+ }
+
+ public void setConf(Configuration conf) {
+ this.conf = conf;
+ }
+
+ public Abortable getAbort() {
+ return abort;
+ }
+
+ public void setAbort(Abortable abort) {
+ this.abort = abort;
+ }
+}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesHBaseImpl.java hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesHBaseImpl.java
new file mode 100644
index 0000000..29f0632
--- /dev/null
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesHBaseImpl.java
@@ -0,0 +1,497 @@
+/*
+*
+* 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.replication;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
+import sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.SortedSet;
+
+@InterfaceAudience.Private
+public class ReplicationQueuesHBaseImpl implements ReplicationQueues {
+
+ /** Name of the HBase Table used for tracking replication*/
+ public static final TableName REPLICATION_TABLE_NAME =
+ TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication");
+
+ // Column family and column names for the Replication Table
+ private static final byte[] CF = Bytes.toBytes("r");
+ private static final byte[] COL_OWNER = Bytes.toBytes("o");
+ private static final byte[] COL_QUEUE_ID = Bytes.toBytes("q");
+
+ // Column Descriptor for the Replication Table
+ private static final HColumnDescriptor REPLICATION_COL_DESCRIPTOR =
+ new HColumnDescriptor(CF).setMaxVersions(1)
+ .setInMemory(true)
+ .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+ // TODO: Figure out which bloom filter to use
+ .setBloomFilterType(BloomType.NONE)
+ .setCacheDataInL1(true);
+
+ // Common byte values used in replication offset tracking
+ private static final byte[] INITIAL_OFFSET = Bytes.toBytes(0L);
+
+ /*
+ * Make sure that HBase table operations for replication have a high number of retries. This is
+ * because the server is aborted if any HBase table operation fails. Each RPC will be attempted
+ * 3600 times before exiting. This provides each operation with 2 hours of retries
+ * before the server is aborted.
+ */
+ private static final int CLIENT_RETRIES = 3600;
+ private static final int RPC_TIMEOUT = 2000;
+ private static final int OPERATION_TIMEOUT = CLIENT_RETRIES * RPC_TIMEOUT;
+
+ private final Configuration conf;
+ private final Admin admin;
+ private final Connection connection;
+ private final Table replicationTable;
+ private final Abortable abortable;
+ private String serverName = null;
+ private byte[] serverNameBytes = null;
+
+ public ReplicationQueuesHBaseImpl(ReplicationQueuesArguments args) throws IOException {
+ this(args.getConf(), args.getAbort());
+ }
+
+ public ReplicationQueuesHBaseImpl(Configuration conf, Abortable abort) throws IOException {
+ this.conf = new Configuration(conf);
+ // Modify the connection's config so that the Replication Table it returns has a much higher
+ // number of client retries
+ conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, CLIENT_RETRIES);
+ this.connection = ConnectionFactory.createConnection(conf);
+ this.admin = connection.getAdmin();
+ this.abortable = abort;
+ replicationTable = createAndGetReplicationTable();
+ replicationTable.setRpcTimeout(RPC_TIMEOUT);
+ replicationTable.setOperationTimeout(OPERATION_TIMEOUT);
+ }
+
+ @Override
+ public void init(String serverName) throws ReplicationException {
+ this.serverName = serverName;
+ this.serverNameBytes = Bytes.toBytes(serverName);
+ }
+
+ @Override
+ public void removeQueue(String queueId) {
+ try {
+ byte[] rowKey = this.queueIdToRowKey(queueId);
+ // The rowkey will be null if the queue cannot be found in the Replication Table
+ if (rowKey == null) {
+ String errMsg = "Could not remove non-existent queue with queueId=" + queueId;
+ abortable.abort(errMsg, new ReplicationException(errMsg));
+ return;
+ }
+ Delete deleteQueue = new Delete(rowKey);
+ safeQueueUpdate(deleteQueue);
+ } catch (IOException e) {
+ abortable.abort("Could not remove queue with queueId=" + queueId, e);
+ }
+ }
+
+ @Override
+ public void addLog(String queueId, String filename) throws ReplicationException {
+ try {
+ // Check if the queue info (Owner, QueueId) is currently stored in the Replication Table
+ if (this.queueIdToRowKey(queueId) == null) {
+ // Each queue will have an Owner, QueueId, and a collection of [WAL:offset] key values.
+ Put putNewQueue = new Put(Bytes.toBytes(buildServerQueueName(queueId)));
+ putNewQueue.addColumn(CF, COL_OWNER, Bytes.toBytes(serverName));
+ putNewQueue.addColumn(CF, COL_QUEUE_ID, Bytes.toBytes(queueId));
+ putNewQueue.addColumn(CF, Bytes.toBytes(filename), INITIAL_OFFSET);
+ replicationTable.put(putNewQueue);
+ } else {
+ // Otherwise simply add the new log and offset as a new column
+ Put putNewLog = new Put(this.queueIdToRowKey(queueId));
+ putNewLog.addColumn(CF, Bytes.toBytes(filename), INITIAL_OFFSET);
+ safeQueueUpdate(putNewLog);
+ }
+ } catch (IOException e) {
+ abortable.abort("Could not add queue queueId=" + queueId + " filename=" + filename, e);
+ }
+ }
+
+ @Override
+ public void removeLog(String queueId, String filename) {
+ try {
+ byte[] rowKey = this.queueIdToRowKey(queueId);
+ if (rowKey == null) {
+ String errMsg = "Could not remove log from non-existent queueId=" + queueId + ", filename="
+ + filename;
+ abortable.abort(errMsg, new ReplicationException(errMsg));
+ return;
+ }
+ Delete delete = new Delete(rowKey);
+ delete.addColumns(CF, Bytes.toBytes(filename));
+ safeQueueUpdate(delete);
+ } catch (IOException e) {
+ abortable.abort("Could not remove log from queueId=" + queueId + ", filename=" + filename, e);
+ }
+ }
+
+ @Override
+ public void setLogPosition(String queueId, String filename, long position) {
+ try {
+ byte[] rowKey = this.queueIdToRowKey(queueId);
+ if (rowKey == null) {
+ String errMsg = "Could not set position of log from non-existent queueId=" + queueId +
+ ", filename=" + filename;
+ abortable.abort(errMsg, new ReplicationException(errMsg));
+ return;
+ }
+ // Check that the log exists. addLog() must have been called before setLogPosition().
+ Get checkLogExists = new Get(rowKey);
+ checkLogExists.addColumn(CF, Bytes.toBytes(filename));
+ if (!replicationTable.exists(checkLogExists)) {
+ String errMsg = "Could not set position of non-existent log from queueId=" + queueId +
+ ", filename=" + filename;
+ abortable.abort(errMsg, new ReplicationException(errMsg));
+ return;
+ }
+ // Update the log offset if it exists
+ Put walAndOffset = new Put(rowKey);
+ walAndOffset.addColumn(CF, Bytes.toBytes(filename), Bytes.toBytes(position));
+ safeQueueUpdate(walAndOffset);
+ } catch (IOException e) {
+ abortable.abort("Failed to write replication wal position (filename=" + filename +
+ ", position=" + position + ")", e);
+ }
+ }
+
+ @Override
+ public long getLogPosition(String queueId, String filename) throws ReplicationException {
+ try {
+ byte[] rowKey = this.queueIdToRowKey(queueId);
+ if (rowKey == null) {
+ throw new ReplicationException("Could not get position in log for non-existent queue " +
+ "queueId=" + queueId + ", filename=" + filename);
+ }
+ Get getOffset = new Get(rowKey);
+ getOffset.addColumn(CF, Bytes.toBytes(filename));
+ Result result = replicationTable.get(getOffset);
+ if (result.isEmpty()) {
+ throw new ReplicationException("Could not read empty result while getting log position " +
+ "queueId=" + queueId + ", filename=" + filename);
+ }
+ return Bytes.toLong(result.getValue(CF, Bytes.toBytes(filename)));
+ } catch (IOException e) {
+ throw new ReplicationException("Could not get position in log for queueId=" + queueId +
+ ", filename=" + filename);
+ }
+ }
+
+ @Override
+ public void removeAllQueues() {
+ List myQueueIds = getAllQueues();
+ for (String queueId : myQueueIds) {
+ removeQueue(queueId);
+ }
+ }
+
+ @Override
+ public List getLogsInQueue(String queueId) {
+ List logs = new ArrayList();
+ try {
+ byte[] rowKey = this.queueIdToRowKey(queueId);
+ if (rowKey == null) {
+ String errMsg = "Could not get logs from non-existent queueId=" + queueId;
+ abortable.abort(errMsg, new ReplicationException(errMsg));
+ return null;
+ }
+ Get getQueue = new Get(rowKey);
+ Result queue = replicationTable.get(getQueue);
+ if (queue.isEmpty()) {
+ return null;
+ }
+ Map familyMap = queue.getFamilyMap(CF);
+ for (byte[] cQualifier : familyMap.keySet()) {
+ if (Arrays.equals(cQualifier, COL_OWNER) || Arrays.equals(cQualifier, COL_QUEUE_ID)) {
+ continue;
+ }
+ logs.add(Bytes.toString(cQualifier));
+ }
+ } catch (IOException e) {
+ abortable.abort("Could not get logs from queue queueId=" + queueId, e);
+ return null;
+ }
+ return logs;
+ }
+
+ @Override
+ public List getAllQueues() {
+ try {
+ return this.getQueuesBelongingToServer(serverName);
+ } catch (IOException e) {
+ abortable.abort("Could not get all replication queues", e);
+ return null;
+ }
+ }
+
+ @Override
+ public SortedMap> claimQueues(String regionserver) {
+ // TODO
+ throw new NotImplementedException();
+ }
+
+ @Override
+ public List getListOfReplicators() {
+ // TODO
+ throw new NotImplementedException();
+ }
+
+ @Override
+ public boolean isThisOurRegionServer(String regionserver) {
+ return this.serverName.equals(regionserver);
+ }
+
+ @Override
+ public void addPeerToHFileRefs(String peerId) throws ReplicationException {
+ // TODO
+ throw new NotImplementedException();
+ }
+
+ @Override
+ public void removePeerFromHFileRefs(String peerId) {
+ // TODO
+ throw new NotImplementedException();
+ }
+
+ @Override
+ public void addHFileRefs(String peerId, List files) throws ReplicationException {
+ // TODO
+ throw new NotImplementedException();
+ }
+
+ @Override
+ public void removeHFileRefs(String peerId, List files) {
+ // TODO
+ throw new NotImplementedException();
+ }
+
+ /**
+ * Gets the Replication Table. Builds and blocks until the table is available if the Replication
+ * Table does not exist.
+ *
+ * @return the Replication Table
+ * @throws IOException if the Replication Table takes too long to build
+ */
+ private Table createAndGetReplicationTable() throws IOException {
+ if (!replicationTableExists()) {
+ createReplicationTable();
+ }
+ int maxRetries = conf.getInt("replication.queues.createtable.retries.number", 100);
+ RetryCounterFactory counterFactory = new RetryCounterFactory(maxRetries, 100);
+ RetryCounter retryCounter = counterFactory.create();
+ while (!replicationTableExists()) {
+ try {
+ retryCounter.sleepUntilNextRetry();
+ if (!retryCounter.shouldRetry()) {
+ throw new IOException("Unable to acquire the Replication Table");
+ }
+ } catch (InterruptedException e) {
+ return null;
+ }
+ }
+ return connection.getTable(REPLICATION_TABLE_NAME);
+ }
+
+ /**
+ * Checks whether the Replication Table exists yet
+ *
+ * @return whether the Replication Table exists
+ * @throws IOException
+ */
+ private boolean replicationTableExists() {
+ try {
+ return admin.tableExists(REPLICATION_TABLE_NAME);
+ } catch (IOException e) {
+ return false;
+ }
+ }
+
+ /**
+ * Create the replication table with the provided HColumnDescriptor REPLICATION_COL_DESCRIPTOR
+ * in ReplicationQueuesHBaseImpl
+ * @throws IOException
+ */
+ private void createReplicationTable() throws IOException {
+ HTableDescriptor replicationTableDescriptor = new HTableDescriptor(REPLICATION_TABLE_NAME);
+ replicationTableDescriptor.addFamily(REPLICATION_COL_DESCRIPTOR);
+ admin.createTable(replicationTableDescriptor);
+ }
+
+ /**
+ * Builds the unique identifier for a queue in the Replication table by appending the queueId to
+ * the servername
+ *
+ * @param queueId a String that identifies the queue
+ * @return unique identifier for a queue in the Replication table
+ */
+ private String buildServerQueueName(String queueId) {
+ return serverName + "-" + queueId;
+ }
+
+ /**
+ * See safeQueueUpdate(RowMutations mutate)
+ * @param put Row mutation to perform on the queue
+ */
+ private void safeQueueUpdate(Put put) {
+ RowMutations mutations = new RowMutations(put.getRow());
+ try {
+ mutations.add(put);
+ } catch (IOException e){
+ abortable.abort("Failed to update Replication Table because of IOException", e);
+ }
+ safeQueueUpdate(mutations);
+ }
+
+ /**
+ * See safeQueueUpdate(RowMutations mutate)
+ * @param delete Row mutation to perform on the queue
+ */
+ private void safeQueueUpdate(Delete delete) {
+ RowMutations mutations = new RowMutations(delete.getRow());
+ try {
+ mutations.add(delete);
+ } catch (IOException e) {
+ abortable.abort("Failed to update Replication Table because of IOException", e);
+ }
+ safeQueueUpdate(mutations);
+ }
+
+ /**
+ * Attempt to mutate a given queue in the Replication Table with a checkAndPut on the OWNER column
+ * of the queue. Abort the server if this checkAndPut fails: which means we have somehow lost
+ * ownership of the column or an IO Exception has occurred during the transaction.
+ *
+ * @param mutate Mutation to perform on a given queue
+ */
+ private void safeQueueUpdate(RowMutations mutate) {
+ try {
+ boolean updateSuccess = replicationTable.checkAndMutate(mutate.getRow(), CF, COL_OWNER,
+ CompareFilter.CompareOp.EQUAL, serverNameBytes, mutate);
+ if (!updateSuccess) {
+ String errMsg = "Failed to update Replication Table because we lost queue ownership";
+ abortable.abort(errMsg, new ReplicationException(errMsg));
+ }
+ } catch (IOException e) {
+ abortable.abort("Failed to update Replication Table because of IOException", e);
+ }
+ }
+
+ /**
+ * Get the QueueIds belonging to the named server from the ReplicationTable
+ *
+ * @param server name of the server
+ * @return a list of the QueueIds belonging to the server
+ * @throws IOException
+ */
+ private List getQueuesBelongingToServer(String server) throws IOException {
+ List queues = new ArrayList();
+ Scan scan = new Scan();
+ SingleColumnValueFilter filterMyQueues = new SingleColumnValueFilter(CF, COL_OWNER,
+ CompareFilter.CompareOp.EQUAL, Bytes.toBytes(server));
+ scan.setFilter(filterMyQueues);
+ scan.addColumn(CF, COL_QUEUE_ID);
+ scan.addColumn(CF, COL_OWNER);
+ ResultScanner results = replicationTable.getScanner(scan);
+ for (Result result : results) {
+ queues.add(Bytes.toString(result.getValue(CF, COL_QUEUE_ID)));
+ }
+ results.close();
+ return queues;
+ }
+
+ /**
+ * Finds the row key of the HBase row corresponding to the provided queue. This has to be done,
+ * because the row key is [original server name + "-" + queueId0]. And the original server will
+ * make calls to getLog(), getQueue(), etc. with the argument queueId = queueId0.
+ * On the original server we can build the row key by concatenating servername + queueId0.
+ * Yet if the queue is claimed by another server, future calls to getLog(), getQueue(), etc.
+ * will be made with the argument queueId = queueId0 + "-" + pastOwner0 + "-" + pastOwner1 ...
+ * so we need a way to look up rows by their modified queueId's.
+ *
+ * TODO: Consider updating the queueId passed to getLog, getQueue()... inside of ReplicationSource
+ * TODO: and ReplicationSourceManager or the parsing of the passed in queueId's so that we don't
+ * TODO have to scan the table for row keys for each update. See HBASE-15956.
+ *
+ * TODO: We can also cache queueId's if ReplicationQueuesHBaseImpl becomes a bottleneck. We
+ * TODO: currently perform scan's over all the rows looking for one with a matching QueueId.
+ *
+ * @param queueId string representation of the queue id
+ * @return the rowkey of the corresponding queue. This returns null if the corresponding queue
+ * cannot be found.
+ * @throws IOException
+ */
+ private byte[] queueIdToRowKey(String queueId) throws IOException {
+ Scan scan = new Scan();
+ scan.addColumn(CF, COL_QUEUE_ID);
+ scan.addColumn(CF, COL_OWNER);
+ scan.setMaxResultSize(1);
+ // Search for the queue that matches this queueId
+ SingleColumnValueFilter filterByQueueId = new SingleColumnValueFilter(CF, COL_QUEUE_ID,
+ CompareFilter.CompareOp.EQUAL, Bytes.toBytes(queueId));
+ // Make sure that we are the owners of the queue. QueueId's may overlap.
+ SingleColumnValueFilter filterByOwner = new SingleColumnValueFilter(CF, COL_OWNER,
+ CompareFilter.CompareOp.EQUAL, Bytes.toBytes(serverName));
+ // We only want the row key
+ FirstKeyOnlyFilter filterOutColumns = new FirstKeyOnlyFilter();
+ FilterList filterList = new FilterList(filterByQueueId, filterByOwner, filterOutColumns);
+ scan.setFilter(filterList);
+ ResultScanner results = replicationTable.getScanner(scan);
+ Result result = results.next();
+ results.close();
+ return (result == null) ? null : result.getRow();
+ }
+}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 2bb8ea8..f03efff 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -41,7 +41,8 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
- * This class provides an implementation of the ReplicationQueues interface using ZooKeeper. The
+ * This class provides an implementation of the
+ * interface using ZooKeeper. The
* base znode that this class works at is the myQueuesZnode. The myQueuesZnode contains a list of
* all outstanding WAL files on this region server that need to be replicated. The myQueuesZnode is
* the regionserver name (a concatenation of the region server’s hostname, client port and start
@@ -71,6 +72,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
private static final Log LOG = LogFactory.getLog(ReplicationQueuesZKImpl.class);
+ public ReplicationQueuesZKImpl(ReplicationQueuesArguments args) {
+ this(args.getZk(), args.getConf(), args.getAbort());
+ }
+
public ReplicationQueuesZKImpl(final ZooKeeperWatcher zk, Configuration conf,
Abortable abortable) {
super(zk, conf, abortable);
@@ -84,12 +89,14 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
} catch (KeeperException e) {
throw new ReplicationException("Could not initialize replication queues.", e);
}
- // Irrespective of bulk load hfile replication is enabled or not we add peerId node to
- // hfile-refs node -- HBASE-15397
- try {
- ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
- } catch (KeeperException e) {
- throw new ReplicationException("Could not initialize hfile references replication queue.", e);
+ if (conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+ HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT)) {
+ try {
+ ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
+ } catch (KeeperException e) {
+ throw new ReplicationException("Could not initialize hfile references replication queue.",
+ e);
+ }
}
}
@@ -166,8 +173,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
}
@Override
- public boolean isThisOurZnode(String znode) {
- return ZKUtil.joinZNode(this.queuesZNode, znode).equals(this.myQueuesZnode);
+ public boolean isThisOurRegionServer(String regionserver) {
+ return ZKUtil.joinZNode(this.queuesZNode, regionserver).equals(this.myQueuesZnode);
}
@Override
@@ -223,7 +230,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
this.abortable.abort("Failed to get a list of queues for region server: "
+ this.myQueuesZnode, e);
}
- return listOfQueues;
+ return listOfQueues == null ? new ArrayList() : listOfQueues;
}
/**
@@ -499,4 +506,23 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
e);
}
}
+
+ @Override
+ public void removePeerFromHFileRefs(String peerId) {
+ final String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
+ try {
+ if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Peer " + peerZnode + " not found in hfile reference queue.");
+ }
+ return;
+ } else {
+ LOG.info("Removing peer " + peerZnode + " from hfile reference queue.");
+ ZKUtil.deleteNodeRecursively(this.zookeeper, peerZnode);
+ }
+ } catch (KeeperException e) {
+ LOG.error("Ignoring the exception to remove peer " + peerId + " from hfile reference queue.",
+ e);
+ }
+ }
}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java
index bfb625b..06d01d7 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslClientHandler.java
@@ -39,6 +39,7 @@ import javax.security.sasl.SaslClient;
import javax.security.sasl.SaslException;
import java.io.IOException;
+import java.nio.ByteBuffer;
import java.nio.charset.Charset;
import java.security.PrivilegedExceptionAction;
import java.util.Map;
@@ -63,6 +64,7 @@ public class SaslClientHandler extends ChannelDuplexHandler {
private final SaslExceptionHandler exceptionHandler;
private final SaslSuccessfulConnectHandler successfulConnectHandler;
private byte[] saslToken;
+ private byte[] connectionHeader;
private boolean firstRead = true;
private int retryCount = 0;
@@ -80,10 +82,11 @@ public class SaslClientHandler extends ChannelDuplexHandler {
*/
public SaslClientHandler(UserGroupInformation ticket, AuthMethod method,
Token extends TokenIdentifier> token, String serverPrincipal, boolean fallbackAllowed,
- String rpcProtection, SaslExceptionHandler exceptionHandler,
+ String rpcProtection, byte[] connectionHeader, SaslExceptionHandler exceptionHandler,
SaslSuccessfulConnectHandler successfulConnectHandler) throws IOException {
this.ticket = ticket;
this.fallbackAllowed = fallbackAllowed;
+ this.connectionHeader = connectionHeader;
this.exceptionHandler = exceptionHandler;
this.successfulConnectHandler = successfulConnectHandler;
@@ -225,8 +228,13 @@ public class SaslClientHandler extends ChannelDuplexHandler {
if (!useWrap) {
ctx.pipeline().remove(this);
+ successfulConnectHandler.onSuccess(ctx.channel());
+ } else {
+ byte[] wrappedCH = saslClient.wrap(connectionHeader, 0, connectionHeader.length);
+ // write connection header
+ writeSaslToken(ctx, wrappedCH);
+ successfulConnectHandler.onSaslProtectionSucess(ctx.channel());
}
- successfulConnectHandler.onSuccess(ctx.channel());
}
}
// Normal wrapped reading
@@ -303,9 +311,11 @@ public class SaslClientHandler extends ChannelDuplexHandler {
super.write(ctx, msg, promise);
} else {
ByteBuf in = (ByteBuf) msg;
+ byte[] unwrapped = new byte[in.readableBytes()];
+ in.readBytes(unwrapped);
try {
- saslToken = saslClient.wrap(in.array(), in.readerIndex(), in.readableBytes());
+ saslToken = saslClient.wrap(unwrapped, 0, unwrapped.length);
} catch (SaslException se) {
try {
saslClient.dispose();
@@ -355,5 +365,12 @@ public class SaslClientHandler extends ChannelDuplexHandler {
* @param channel which is successfully authenticated
*/
public void onSuccess(Channel channel);
+
+ /**
+ * Runs on success if data protection used in Sasl
+ *
+ * @param channel which is successfully authenticated
+ */
+ public void onSaslProtectionSucess(Channel channel);
}
}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
index 25ac01f..79dbd05 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
@@ -217,11 +217,16 @@ public class AccessControlClient {
HTableDescriptor[] htds = null;
if (tableRegex == null || tableRegex.isEmpty()) {
permList = ProtobufUtil.getUserPermissions(controller, protocol);
- } else if (tableRegex.charAt(0) == '@') {
- String namespace = tableRegex.substring(1);
- permList = ProtobufUtil.getUserPermissions(controller, protocol,
- Bytes.toBytes(namespace));
- } else {
+ } else if (tableRegex.charAt(0) == '@') { // Namespaces
+ String namespaceRegex = tableRegex.substring(1);
+ for (NamespaceDescriptor nsds : admin.listNamespaceDescriptors()) { // Read out all namespaces
+ String namespace = nsds.getName();
+ if (namespace.matches(namespaceRegex)) { // Match the given namespace regex?
+ permList.addAll(ProtobufUtil.getUserPermissions(controller, protocol,
+ Bytes.toBytes(namespace)));
+ }
+ }
+ } else { // Tables
htds = admin.listTables(Pattern.compile(tableRegex), true);
for (HTableDescriptor hd : htds) {
permList.addAll(ProtobufUtil.getUserPermissions(controller, protocol,
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
index 9d0319b..b683fcc 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
@@ -23,6 +23,7 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
+import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
@@ -254,7 +255,7 @@ public class PoolMap implements Map {
}
public static String fuzzyNormalize(String name) {
- return name != null ? name.replaceAll("-", "").trim().toLowerCase() : "";
+ return name != null ? name.replaceAll("-", "").trim().toLowerCase(Locale.ROOT) : "";
}
public static PoolType fuzzyMatch(String name) {
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
index 0b53f95..497e8c4 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -17,15 +17,18 @@
*/
package org.apache.hadoop.hbase.zookeeper;
+import com.google.protobuf.InvalidProtocolBufferException;
+
import java.io.EOFException;
import java.io.IOException;
import java.net.ConnectException;
import java.net.NoRouteToHostException;
import java.net.SocketException;
import java.net.SocketTimeoutException;
-import java.rmi.UnknownHostException;
+import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.List;
+import java.util.Locale;
import org.apache.commons.logging.Log;
@@ -37,8 +40,6 @@ import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -58,14 +59,12 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.KeeperException;
-import com.google.protobuf.InvalidProtocolBufferException;
-
/**
* Utility class to perform operation (get/wait for/verify/set/delete) on znode in ZooKeeper
* which keeps hbase:meta region server location.
*
* Stateless class with a bunch of static methods. Doesn't manage resources passed in
- * (e.g. HConnection, ZooKeeperWatcher etc).
+ * (e.g. Connection, ZooKeeperWatcher etc).
*
* Meta region location is set by RegionServerServices.
* This class doesn't use ZK watchers, rather accesses ZK directly.
@@ -259,7 +258,7 @@ public class MetaTableLocator {
* @throws java.io.IOException
* @throws InterruptedException
*/
- public boolean verifyMetaRegionLocation(HConnection hConnection,
+ public boolean verifyMetaRegionLocation(ClusterConnection hConnection,
ZooKeeperWatcher zkw, final long timeout)
throws InterruptedException, IOException {
return verifyMetaRegionLocation(hConnection, zkw, timeout, HRegionInfo.DEFAULT_REPLICA_ID);
@@ -267,7 +266,7 @@ public class MetaTableLocator {
/**
* Verify hbase:meta is deployed and accessible.
- * @param hConnection
+ * @param connection
* @param zkw
* @param timeout How long to wait on zk for meta address (passed through to
* @param replicaId
@@ -275,12 +274,12 @@ public class MetaTableLocator {
* @throws InterruptedException
* @throws IOException
*/
- public boolean verifyMetaRegionLocation(HConnection hConnection,
+ public boolean verifyMetaRegionLocation(ClusterConnection connection,
ZooKeeperWatcher zkw, final long timeout, int replicaId)
throws InterruptedException, IOException {
AdminProtos.AdminService.BlockingInterface service = null;
try {
- service = getMetaServerConnection(hConnection, zkw, timeout, replicaId);
+ service = getMetaServerConnection(connection, zkw, timeout, replicaId);
} catch (NotAllMetaRegionsOnlineException e) {
// Pass
} catch (ServerNotRunningYetException e) {
@@ -290,7 +289,7 @@ public class MetaTableLocator {
} catch (RegionServerStoppedException e) {
// Pass -- server name sends us to a server that is dying or already dead.
}
- return (service != null) && verifyRegionLocation(hConnection, service,
+ return (service != null) && verifyRegionLocation(connection, service,
getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
HRegionInfo.FIRST_META_REGIONINFO, replicaId).getRegionName());
}
@@ -310,7 +309,7 @@ public class MetaTableLocator {
// rather than have to pass it in. Its made awkward by the fact that the
// HRI is likely a proxy against remote server so the getServerName needs
// to be fixed to go to a local method or to a cache before we can do this.
- private boolean verifyRegionLocation(final Connection connection,
+ private boolean verifyRegionLocation(final ClusterConnection connection,
AdminService.BlockingInterface hostingServer, final ServerName address,
final byte [] regionName)
throws IOException {
@@ -319,10 +318,7 @@ public class MetaTableLocator {
return false;
}
Throwable t;
- PayloadCarryingRpcController controller = null;
- if (connection instanceof ClusterConnection) {
- controller = ((ClusterConnection) connection).getRpcControllerFactory().newController();
- }
+ PayloadCarryingRpcController controller = connection.getRpcControllerFactory().newController();
try {
// Try and get regioninfo from the hosting server.
return ProtobufUtil.getRegionInfo(controller, hostingServer, regionName) != null;
@@ -353,7 +349,7 @@ public class MetaTableLocator {
* Gets a connection to the server hosting meta, as reported by ZooKeeper,
* waiting up to the specified timeout for availability.
* WARNING: Does not retry. Use an {@link org.apache.hadoop.hbase.client.HTable} instead.
- * @param hConnection
+ * @param connection
* @param zkw
* @param timeout How long to wait on meta location
* @param replicaId
@@ -362,10 +358,10 @@ public class MetaTableLocator {
* @throws NotAllMetaRegionsOnlineException if timed out waiting
* @throws IOException
*/
- private AdminService.BlockingInterface getMetaServerConnection(HConnection hConnection,
+ private AdminService.BlockingInterface getMetaServerConnection(ClusterConnection connection,
ZooKeeperWatcher zkw, long timeout, int replicaId)
throws InterruptedException, NotAllMetaRegionsOnlineException, IOException {
- return getCachedConnection(hConnection, waitMetaRegionLocation(zkw, replicaId, timeout));
+ return getCachedConnection(connection, waitMetaRegionLocation(zkw, replicaId, timeout));
}
/**
@@ -376,7 +372,7 @@ public class MetaTableLocator {
* @throws IOException
*/
@SuppressWarnings("deprecation")
- private static AdminService.BlockingInterface getCachedConnection(HConnection hConnection,
+ private static AdminService.BlockingInterface getCachedConnection(ClusterConnection connection,
ServerName sn)
throws IOException {
if (sn == null) {
@@ -384,7 +380,7 @@ public class MetaTableLocator {
}
AdminService.BlockingInterface service = null;
try {
- service = hConnection.getAdmin(sn);
+ service = connection.getAdmin(sn);
} catch (RetriesExhaustedException e) {
if (e.getCause() != null && e.getCause() instanceof ConnectException) {
// Catch this; presume it means the cached connection has gone bad.
@@ -410,7 +406,7 @@ public class MetaTableLocator {
} else if (cause != null && cause instanceof EOFException) {
// Catch. Other end disconnected us.
} else if (cause != null && cause.getMessage() != null &&
- cause.getMessage().toLowerCase().contains("connection reset")) {
+ cause.getMessage().toLowerCase(Locale.ROOT).contains("connection reset")) {
// Catch. Connection reset.
} else {
throw ioe;
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index ae26400..f5b720e 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -1926,27 +1926,27 @@ public class ZKUtil {
int port = sp.length > 1 ? Integer.parseInt(sp[1])
: HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
- Socket socket = new Socket();
InetSocketAddress sockAddr = new InetSocketAddress(host, port);
- socket.connect(sockAddr, timeout);
-
- socket.setSoTimeout(timeout);
- PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
- BufferedReader in = new BufferedReader(new InputStreamReader(
- socket.getInputStream()));
- out.println("stat");
- out.flush();
- ArrayList res = new ArrayList();
- while (true) {
- String line = in.readLine();
- if (line != null) {
- res.add(line);
- } else {
- break;
+ try (Socket socket = new Socket()) {
+ socket.connect(sockAddr, timeout);
+
+ socket.setSoTimeout(timeout);
+ PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
+ BufferedReader in = new BufferedReader(new InputStreamReader(
+ socket.getInputStream()));
+ out.println("stat");
+ out.flush();
+ ArrayList res = new ArrayList();
+ while (true) {
+ String line = in.readLine();
+ if (line != null) {
+ res.add(line);
+ } else {
+ break;
+ }
}
+ return res.toArray(new String[res.size()]);
}
- socket.close();
- return res.toArray(new String[res.size()]);
}
private static void logRetrievedMsg(final ZooKeeperWatcher zkw,
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
index 93828eb..ff3d1c7 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
@@ -171,7 +171,17 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, pendingWatcher, identifier);
pendingWatcher.prepare(this);
if (canCreateBaseZNode) {
- createBaseZNodes();
+ try {
+ createBaseZNodes();
+ } catch (ZooKeeperConnectionException zce) {
+ try {
+ this.recoverableZooKeeper.close();
+ } catch (InterruptedException ie) {
+ LOG.debug("Encountered InterruptedException when closing " + this.recoverableZooKeeper);
+ Thread.currentThread().interrupt();
+ }
+ throw zce;
+ }
}
}
diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
index 680f2c1..70380e6 100644
--- hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
+++ hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
@@ -55,35 +55,36 @@ public class TestHTableDescriptor {
public void testAddCoprocessorWithSpecStr() throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
String cpName = "a.b.c.d";
- boolean expected = false;
try {
htd.addCoprocessorWithSpec(cpName);
+ fail();
} catch (IllegalArgumentException iae) {
- expected = true;
+ // Expected as cpName is invalid
}
- if (!expected) fail();
+
// Try minimal spec.
try {
htd.addCoprocessorWithSpec("file:///some/path" + "|" + cpName);
+ fail();
} catch (IllegalArgumentException iae) {
- expected = false;
+ // Expected to be invalid
}
- if (expected) fail();
+
// Try more spec.
String spec = "hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2";
try {
htd.addCoprocessorWithSpec(spec);
} catch (IllegalArgumentException iae) {
- expected = false;
+ fail();
}
- if (expected) fail();
+
// Try double add of same coprocessor
try {
htd.addCoprocessorWithSpec(spec);
+ fail();
} catch (IOException ioe) {
- expected = true;
+ // Expect that the coprocessor already exists
}
- if (!expected) fail();
}
@Test
diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
index 0e0fbb0..426b6a7 100644
--- hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
+++ hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
@@ -20,7 +20,11 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import java.lang.annotation.Annotation;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
@@ -28,6 +32,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Triple;
import org.apache.hadoop.hbase.ClassFinder.And;
import org.apache.hadoop.hbase.ClassFinder.FileNameFilter;
import org.apache.hadoop.hbase.ClassFinder.Not;
@@ -59,6 +65,7 @@ import org.junit.experimental.categories.Category;
@Category(SmallTests.class)
public class TestInterfaceAudienceAnnotations {
+ private static final String HBASE_PROTOBUF = "org.apache.hadoop.hbase.protobuf.generated";
private static final Log LOG = LogFactory.getLog(TestInterfaceAudienceAnnotations.class);
/** Selects classes with generated in their package name */
@@ -180,6 +187,28 @@ public class TestInterfaceAudienceAnnotations {
c.equals(InterfaceStability.Evolving.class);
}
+ private boolean isInterfacePrivateMethod(Method m) {
+ if(m.getDeclaredAnnotations().length > 0) {
+ for(Annotation ann : m.getDeclaredAnnotations()) {
+ if(ann.annotationType().equals(InterfaceAudience.Private.class)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ private boolean isInterfacePrivateContructor(Constructor> c) {
+ if(c.getDeclaredAnnotations().length > 0) {
+ for(Annotation ann : c.getDeclaredAnnotations()) {
+ if(ann.annotationType().equals(InterfaceAudience.Private.class)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
/** Selects classes that are declared public */
class PublicClassFilter implements ClassFinder.ClassFilter {
@Override
@@ -299,4 +328,127 @@ public class TestInterfaceAudienceAnnotations {
+ "have @InterfaceStability annotation as well",
0, classes.size());
}
+
+ @Test
+ public void testProtosInReturnTypes() throws ClassNotFoundException, IOException, LinkageError {
+ Set> classes = findPublicClasses();
+ List, Method>> protosReturnType = new ArrayList, Method>>();
+ for (Class> clazz : classes) {
+ findProtoInReturnType(clazz, protosReturnType);
+ }
+ if (protosReturnType.size() != 0) {
+ LOG.info("These are the methods that have Protos as the return type");
+ for (Pair, Method> pair : protosReturnType) {
+ LOG.info(pair.getFirst().getName() + " " + pair.getSecond().getName() + " "
+ + pair.getSecond().getReturnType().getName());
+ }
+ }
+
+ Assert.assertEquals("Public exposed methods should not have protos in return type", 0,
+ protosReturnType.size());
+ }
+
+ private Set> findPublicClasses()
+ throws ClassNotFoundException, IOException, LinkageError {
+ ClassFinder classFinder =
+ new ClassFinder(new And(new MainCodeResourcePathFilter(), new TestFileNameFilter()),
+ new Not((FileNameFilter) new TestFileNameFilter()),
+ new And(new PublicClassFilter(), new Not(new TestClassFilter()),
+ new Not(new GeneratedClassFilter()),
+ new InterfaceAudiencePublicAnnotatedClassFilter()));
+ Set> classes = classFinder.findClasses(false);
+ return classes;
+ }
+
+ @Test
+ public void testProtosInParamTypes() throws ClassNotFoundException, IOException, LinkageError {
+ Set> classes = findPublicClasses();
+ List, Method, Class>>> protosParamType =
+ new ArrayList, Method, Class>>>();
+ for (Class> clazz : classes) {
+ findProtoInParamType(clazz, protosParamType);
+ }
+
+ if (protosParamType.size() != 0) {
+ LOG.info("These are the methods that have Protos as the param type");
+ for (Triple, Method, Class>> pair : protosParamType) {
+ LOG.info(pair.getFirst().getName() + " " + pair.getSecond().getName() + " "
+ + pair.getThird().getName());
+ }
+ }
+
+ Assert.assertEquals("Public exposed methods should not have protos in param type", 0,
+ protosParamType.size());
+ }
+
+ @Test
+ public void testProtosInConstructors() throws ClassNotFoundException, IOException, LinkageError {
+ Set> classes = findPublicClasses();
+ List> classList = new ArrayList>();
+ for (Class> clazz : classes) {
+ Constructor>[] constructors = clazz.getConstructors();
+ for (Constructor> cons : constructors) {
+ if (!isInterfacePrivateContructor(cons)) {
+ Class>[] parameterTypes = cons.getParameterTypes();
+ for (Class> param : parameterTypes) {
+ if (param.getName().contains(HBASE_PROTOBUF)) {
+ classList.add(clazz);
+ break;
+ }
+ }
+ }
+ }
+ }
+
+ if (classList.size() != 0) {
+ LOG.info("These are the classes that have Protos in the constructor");
+ for (Class> clazz : classList) {
+ LOG.info(clazz.getName());
+ }
+ }
+
+ Assert.assertEquals("Public exposed classes should not have protos in constructors", 0,
+ classList.size());
+ }
+
+ private void findProtoInReturnType(Class> clazz,
+ List, Method>> protosReturnType) {
+ Pair, Method> returnTypePair = new Pair, Method>();
+ Method[] methods = clazz.getMethods();
+ returnTypePair.setFirst(clazz);
+ for (Method method : methods) {
+ if (clazz.isInterface() || method.getModifiers() == Modifier.PUBLIC) {
+ if (!isInterfacePrivateMethod(method)) {
+ Class> returnType = method.getReturnType();
+ if (returnType.getName().contains(HBASE_PROTOBUF)) {
+ returnTypePair.setSecond(method);
+ protosReturnType.add(returnTypePair);
+ continue;
+ }
+ }
+ }
+ }
+ }
+
+ private void findProtoInParamType(Class> clazz,
+ List, Method, Class>>> protosParamType) {
+ Triple, Method, Class>> paramType = new Triple, Method, Class>>();
+ Method[] methods = clazz.getMethods();
+ paramType.setFirst(clazz);
+ for (Method method : methods) {
+ if (clazz.isInterface() || method.getModifiers() == Modifier.PUBLIC) {
+ if (!isInterfacePrivateMethod(method)) {
+ Class>[] parameters = method.getParameterTypes();
+ for (Class> param : parameters) {
+ if (param.getName().contains(HBASE_PROTOBUF)) {
+ paramType.setSecond(method);
+ paramType.setThird(param);
+ protosParamType.add(paramType);
+ break;
+ }
+ }
+ }
+ }
+ }
+ }
}
diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index 376c02a..d943316 100644
--- hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -712,15 +712,17 @@ public class TestAsyncProcess {
private void doHTableFailedPut(boolean bufferOn) throws Exception {
ClusterConnection conn = createHConnection();
- HTable ht = new HTable(conn, new BufferedMutatorParams(DUMMY_TABLE));
- MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
- ht.mutator.ap = ap;
+ BufferedMutatorParams bufferParam = new BufferedMutatorParams(DUMMY_TABLE);
if (bufferOn) {
- ht.setWriteBufferSize(1024L * 1024L);
+ bufferParam.writeBufferSize(1024L * 1024L);
} else {
- ht.setWriteBufferSize(0L);
+ bufferParam.writeBufferSize(0L);
}
+ HTable ht = new HTable(conn, bufferParam);
+ MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
+ ht.mutator.ap = ap;
+
Put put = createPut(1, false);
Assert.assertEquals(0L, ht.mutator.currentWriteBufferSize.get());
@@ -1133,16 +1135,17 @@ public class TestAsyncProcess {
}
@Test
- public void testWaitForMaximumCurrentTasks() throws InterruptedException, BrokenBarrierException {
+ public void testWaitForMaximumCurrentTasks() throws Exception {
final AtomicLong tasks = new AtomicLong(0);
final AtomicInteger max = new AtomicInteger(0);
final CyclicBarrier barrier = new CyclicBarrier(2);
+ final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf);
Runnable runnable = new Runnable() {
@Override
public void run() {
try {
barrier.await();
- AsyncProcess.waitForMaximumCurrentTasks(max.get(), tasks, 1);
+ ap.waitForMaximumCurrentTasks(max.get(), tasks, 1, null);
} catch (InterruptedIOException e) {
Assert.fail(e.getMessage());
} catch (InterruptedException e) {
diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
index 0a5a37f..e8135a8 100644
--- hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
+++ hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
@@ -104,7 +104,7 @@ public class TestClientNoCluster extends Configured implements Tool {
@Before
public void setUp() throws Exception {
this.conf = HBaseConfiguration.create();
- // Run my HConnection overrides. Use my little ConnectionImplementation below which
+ // Run my Connection overrides. Use my little ConnectionImplementation below which
// allows me insert mocks and also use my Registry below rather than the default zk based
// one so tests run faster and don't have zk dependency.
this.conf.set("hbase.client.registry.impl", SimpleRegistry.class.getName());
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 45d7c34..b05a520 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -1024,13 +1024,13 @@ public final class HConstants {
* by different set of handlers. For example, HIGH_QOS tagged methods are
* handled by high priority handlers.
*/
- // normal_QOS < QOS_threshold < replication_QOS < replay_QOS < admin_QOS < high_QOS
+ // normal_QOS < replication_QOS < replay_QOS < QOS_threshold < admin_QOS < high_QOS
public static final int NORMAL_QOS = 0;
- public static final int QOS_THRESHOLD = 10;
- public static final int HIGH_QOS = 200;
public static final int REPLICATION_QOS = 5;
public static final int REPLAY_QOS = 6;
+ public static final int QOS_THRESHOLD = 10;
public static final int ADMIN_QOS = 100;
+ public static final int HIGH_QOS = 200;
public static final int SYSTEMTABLE_QOS = HIGH_QOS;
/** Directory under /hbase where archived hfiles are stored */
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
index c0c43ed..52db37b 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
@@ -25,6 +25,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
+import java.util.Locale;
import java.util.regex.Pattern;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -232,7 +233,7 @@ import org.apache.hadoop.hbase.util.Bytes;
*/
static String getServerName(String hostName, int port, long startcode) {
final StringBuilder name = new StringBuilder(hostName.length() + 1 + 5 + 1 + 13);
- name.append(hostName.toLowerCase());
+ name.append(hostName.toLowerCase(Locale.ROOT));
name.append(SERVERNAME_SEPARATOR);
name.append(port);
name.append(SERVERNAME_SEPARATOR);
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
index 821b21f..6dc4190 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
@@ -235,7 +235,34 @@ public final class Compression {
throw new RuntimeException(e);
}
}
- };
+ },
+ BZIP2("bzip2") {
+ // Use base type to avoid compile-time dependencies.
+ private volatile transient CompressionCodec bzipCodec;
+ private transient Object lock = new Object();
+
+ @Override
+ CompressionCodec getCodec(Configuration conf) {
+ if (bzipCodec == null) {
+ synchronized (lock) {
+ if (bzipCodec == null) {
+ bzipCodec = buildCodec(conf);
+ }
+ }
+ }
+ return bzipCodec;
+ }
+
+ private CompressionCodec buildCodec(Configuration conf) {
+ try {
+ Class> externalCodec =
+ getClassLoaderForCodec().loadClass("org.apache.hadoop.io.compress.BZip2Codec");
+ return (CompressionCodec) ReflectionUtils.newInstance(externalCodec, conf);
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ };
private final Configuration conf;
private final String compressName;
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
index 2d58a18..7e3c013 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
@@ -30,6 +30,7 @@ import java.security.KeyStoreException;
import java.security.NoSuchAlgorithmException;
import java.security.UnrecoverableKeyException;
import java.security.cert.CertificateException;
+import java.util.Locale;
import java.util.Properties;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -142,7 +143,7 @@ public class KeyStoreKeyProvider implements KeyProvider {
throw new RuntimeException("KeyProvider scheme should specify KeyStore type");
}
// KeyStore expects instance type specifications in uppercase
- store = KeyStore.getInstance(storeType.toUpperCase());
+ store = KeyStore.getInstance(storeType.toUpperCase(Locale.ROOT));
processParameters(uri);
load(uri);
} catch (URISyntaxException e) {
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/CoprocessorClassLoader.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/CoprocessorClassLoader.java
index 78ecc7e..11016c3 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/util/CoprocessorClassLoader.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/CoprocessorClassLoader.java
@@ -177,8 +177,10 @@ public class CoprocessorClassLoader extends ClassLoaderBase {
if (m.matches()) {
File file = new File(parentDirStr, "." + pathPrefix + "."
+ path.getName() + "." + System.currentTimeMillis() + "." + m.group(1));
- IOUtils.copyBytes(jarFile.getInputStream(entry),
- new FileOutputStream(file), conf, true);
+ try (FileOutputStream outStream = new FileOutputStream(file)) {
+ IOUtils.copyBytes(jarFile.getInputStream(entry),
+ outStream, conf, true);
+ }
file.deleteOnExit();
addURL(file.toURI().toURL());
}
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/Triple.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/Triple.java
index 1438ab7..1de6bee 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/util/Triple.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/Triple.java
@@ -28,6 +28,10 @@ public class Triple {
private A first;
private B second;
private C third;
+ // default constructor
+ public Triple() {
+
+ }
public Triple(A first, B second, C third) {
this.first = first;
diff --git hbase-common/src/main/resources/hbase-default.xml hbase-common/src/main/resources/hbase-default.xml
index 62a6b62..55ac497 100644
--- hbase-common/src/main/resources/hbase-default.xml
+++ hbase-common/src/main/resources/hbase-default.xml
@@ -562,7 +562,7 @@ possible configurations would overwhelm and obscure the important.
hbase.regions.slop
0.001
- Rebalance if any regionserver has average + (average * slop) regions.
+ Rebalance if any regionserver has average + (average * slop) regions.
The default value of this parameter is 0.001 in StochasticLoadBalancer (the default load balancer),
while the default is 0.2 in other load balancers (i.e., SimpleLoadBalancer).
@@ -865,7 +865,7 @@ possible configurations would overwhelm and obscure the important.
Must be a multiple of 1024 else you will run into
'java.io.IOException: Invalid HFile block magic' when you go to read from cache.
If you specify no values here, then you pick up the default bucketsizes set
- in code (See BucketAllocator#DEFAULT_BUCKET_SIZES).
+ in code (See BucketAllocator#DEFAULT_BUCKET_SIZES).
@@ -1132,11 +1132,6 @@ possible configurations would overwhelm and obscure the important.
of servers, so this is most useful for debugging only.
- hbase.online.schema.update.enable
- true
- Set true to enable online schema changes.
-
-
hbase.table.lock.enable
true
Set to true to enable locking the table in zookeeper for schema change operations.
diff --git hbase-common/src/test/java/org/apache/hadoop/hbase/CategoryBasedTimeout.java hbase-common/src/test/java/org/apache/hadoop/hbase/CategoryBasedTimeout.java
index 7f7aafd..8a31ff3 100644
--- hbase-common/src/test/java/org/apache/hadoop/hbase/CategoryBasedTimeout.java
+++ hbase-common/src/test/java/org/apache/hadoop/hbase/CategoryBasedTimeout.java
@@ -32,6 +32,15 @@ import org.junit.rules.Timeout;
*/
public class CategoryBasedTimeout extends Timeout {
+ public static Timeout forClass(Class> clazz) {
+ return CategoryBasedTimeout.builder().withTimeout(clazz).withLookingForStuckThread(true)
+ .build();
+ }
+
+ public static Builder builder() {
+ return new CategoryBasedTimeout.Builder();
+ }
+
@Deprecated
public CategoryBasedTimeout(int millis) {
super(millis);
@@ -45,10 +54,6 @@ public class CategoryBasedTimeout extends Timeout {
super(builder);
}
- public static Builder builder() {
- return new CategoryBasedTimeout.Builder();
- }
-
public static class Builder extends Timeout.Builder {
public Timeout.Builder withTimeout(Class> clazz) {
Annotation annotation = clazz.getAnnotation(Category.class);
diff --git hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
index bb89789..ce57e0f 100644
--- hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
+++ hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
@@ -52,14 +52,16 @@ public interface MetricsHBaseServerSource extends BaseSource {
String TOTAL_CALL_TIME_NAME = "totalCallTime";
String TOTAL_CALL_TIME_DESC = "Total call time, including both queued and processing time.";
String QUEUE_SIZE_NAME = "queueSize";
- String QUEUE_SIZE_DESC = "Number of bytes in the call queues.";
+ String QUEUE_SIZE_DESC = "Number of bytes in the call queues; request has been read and " +
+ "parsed and is waiting to run or is currently being executed.";
String GENERAL_QUEUE_NAME = "numCallsInGeneralQueue";
- String GENERAL_QUEUE_DESC = "Number of calls in the general call queue.";
+ String GENERAL_QUEUE_DESC = "Number of calls in the general call queue; " +
+ "parsed requests waiting in scheduler to be executed";
String PRIORITY_QUEUE_NAME = "numCallsInPriorityQueue";
String REPLICATION_QUEUE_NAME = "numCallsInReplicationQueue";
String REPLICATION_QUEUE_DESC =
- "Number of calls in the replication call queue.";
- String PRIORITY_QUEUE_DESC = "Number of calls in the priority call queue.";
+ "Number of calls in the replication call queue waiting to be run";
+ String PRIORITY_QUEUE_DESC = "Number of calls in the priority call queue waiting to be run";
String NUM_OPEN_CONNECTIONS_NAME = "numOpenConnections";
String NUM_OPEN_CONNECTIONS_DESC = "Number of open connections.";
String NUM_ACTIVE_HANDLER_NAME = "numActiveHandler";
diff --git hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
index 4098e26..76bbb09 100644
--- hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
+++ hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
@@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.ipc;
import java.util.HashMap;
+import java.util.Locale;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -48,7 +49,7 @@ public class MetricsHBaseServerSourceFactoryImpl extends MetricsHBaseServerSourc
source = new MetricsHBaseServerSourceImpl(
context,
METRICS_DESCRIPTION,
- context.toLowerCase(),
+ context.toLowerCase(Locale.ROOT),
context + METRICS_JMX_CONTEXT_SUFFIX, wrap);
//Store back in storage
diff --git hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
index 5503675..4291eb7 100644
--- hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
+++ hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.metrics2.MetricsTag;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import java.util.HashMap;
+import java.util.Locale;
import java.util.Map;
import static org.junit.Assert.*;
@@ -245,6 +246,6 @@ public class MetricsAssertHelperImpl implements MetricsAssertHelper {
}
private String canonicalizeMetricName(String in) {
- return in.toLowerCase().replaceAll("[^A-Za-z0-9 ]", "");
+ return in.toLowerCase(Locale.ROOT).replaceAll("[^A-Za-z0-9 ]", "");
}
}
diff --git hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
index f13102d..b6f1aeb 100644
--- hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
+++ hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase;
import java.io.File;
import java.io.IOException;
+import java.util.Locale;
import java.util.Map;
import org.apache.commons.lang.StringUtils;
@@ -205,7 +206,7 @@ public class HBaseClusterManager extends Configured implements ClusterManager {
@Override
public String getCommand(ServiceType service, Operation op) {
return String.format("%s/bin/hbase-daemon.sh %s %s %s", hbaseHome, confDir,
- op.toString().toLowerCase(), service);
+ op.toString().toLowerCase(Locale.ROOT), service);
}
}
@@ -235,7 +236,7 @@ public class HBaseClusterManager extends Configured implements ClusterManager {
@Override
public String getCommand(ServiceType service, Operation op) {
return String.format("%s/sbin/hadoop-daemon.sh %s %s %s", hadoopHome, confDir,
- op.toString().toLowerCase(), service);
+ op.toString().toLowerCase(Locale.ROOT), service);
}
}
@@ -264,7 +265,7 @@ public class HBaseClusterManager extends Configured implements ClusterManager {
@Override
public String getCommand(ServiceType service, Operation op) {
- return String.format("%s/bin/zkServer.sh %s", zookeeperHome, op.toString().toLowerCase());
+ return String.format("%s/bin/zkServer.sh %s", zookeeperHome, op.toString().toLowerCase(Locale.ROOT));
}
@Override
diff --git hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
index db52c78..8f9b92c 100644
--- hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
+++ hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
@@ -101,11 +101,11 @@ public class IntegrationTestRegionReplicaPerf extends IntegrationTestBase {
* Wraps the invocation of {@link PerformanceEvaluation} in a {@code Callable}.
*/
static class PerfEvalCallable implements Callable {
- private final Queue argv = new LinkedList();
+ private final Queue argv = new LinkedList<>();
private final Admin admin;
public PerfEvalCallable(Admin admin, String argv) {
- // TODO: this API is awkward, should take HConnection, not HBaseAdmin
+ // TODO: this API is awkward, should take Connection, not Admin
this.admin = admin;
this.argv.addAll(Arrays.asList(argv.split(" ")));
LOG.debug("Created PerformanceEvaluationCallable with args: " + argv);
diff --git hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
index 717de17..04a3b05 100644
--- hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
+++ hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
@@ -37,6 +37,7 @@ import javax.xml.ws.http.HTTPException;
import java.io.IOException;
import java.net.URI;
import java.util.HashMap;
+import java.util.Locale;
import java.util.Map;
/**
@@ -274,8 +275,8 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
if (role.get("hostRef").get("hostId").getTextValue().equals(hostId) &&
role.get("type")
.getTextValue()
- .toLowerCase()
- .equals(roleType.toLowerCase())) {
+ .toLowerCase(Locale.ROOT)
+ .equals(roleType.toLowerCase(Locale.ROOT))) {
roleValue = role.get(property).getTextValue();
break;
}
@@ -328,7 +329,7 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
// APIs tend to take commands in lowercase, so convert them to save the trouble later.
@Override
public String toString() {
- return name().toLowerCase();
+ return name().toLowerCase(Locale.ROOT);
}
}
@@ -348,4 +349,4 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
private enum Service {
HBASE, HDFS, MAPREDUCE
}
-}
\ No newline at end of file
+}
diff --git hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
index 848017f..a6b502f 100644
--- hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
+++ hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hbase;
import java.io.IOException;
+import java.util.Locale;
import java.util.Set;
import org.apache.commons.cli.CommandLine;
@@ -111,7 +112,7 @@ public class StripeCompactionsPerformanceEvaluation extends AbstractHBaseTool {
} else {
minValueSize = maxValueSize = Integer.parseInt(valueSize);
}
- String datagen = cmd.getOptionValue(DATAGEN_KEY, "default").toLowerCase();
+ String datagen = cmd.getOptionValue(DATAGEN_KEY, "default").toLowerCase(Locale.ROOT);
if ("default".equals(datagen)) {
dataGen = new MultiThreadedAction.DefaultDataGenerator(
minValueSize, maxValueSize, 1, 1, new byte[][] { COLUMN_FAMILY });
diff --git hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java
index c28f3e6..6c0fbcc 100644
--- hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java
+++ hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.ipc;
+import static org.apache.hadoop.hbase.ipc.RpcClient.SPECIFIC_WRITE_THREAD;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
@@ -41,12 +42,6 @@ import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.codec.Codec;
-import org.apache.hadoop.hbase.ipc.AbstractRpcClient;
-import org.apache.hadoop.hbase.ipc.AsyncRpcClient;
-import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
-import org.apache.hadoop.hbase.ipc.RpcClientImpl;
-import org.apache.hadoop.hbase.ipc.RpcScheduler;
-import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto;
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto;
@@ -290,6 +285,7 @@ public class IntegrationTestRpcClient {
static class SimpleClient extends Thread {
AbstractRpcClient rpcClient;
AtomicBoolean running = new AtomicBoolean(true);
+ AtomicBoolean sending = new AtomicBoolean(false);
AtomicReference exception = new AtomicReference<>(null);
Cluster cluster;
String id;
@@ -319,6 +315,7 @@ public class IntegrationTestRpcClient {
if (address == null) {
throw new IOException("Listener channel is closed");
}
+ sending.set(true);
ret = (EchoResponseProto)
rpcClient.callBlockingMethod(md, null, param, ret, user, address);
} catch (Exception e) {
@@ -340,6 +337,9 @@ public class IntegrationTestRpcClient {
void stopRunning() {
running.set(false);
}
+ boolean isSending() {
+ return sending.get();
+ }
void rethrowException() throws Throwable {
if (exception.get() != null) {
@@ -348,6 +348,29 @@ public class IntegrationTestRpcClient {
}
}
+ /*
+ Test that not started connections are successfully removed from connection pool when
+ rpc client is closing.
+ */
+ @Test (timeout = 30000)
+ public void testRpcWithWriteThread() throws IOException, InterruptedException {
+ LOG.info("Starting test");
+ Cluster cluster = new Cluster(1, 1);
+ cluster.startServer();
+ conf.setBoolean(SPECIFIC_WRITE_THREAD, true);
+ for(int i = 0; i <1000; i++) {
+ AbstractRpcClient rpcClient = createRpcClient(conf, true);
+ SimpleClient client = new SimpleClient(cluster, rpcClient, "Client1");
+ client.start();
+ while(!client.isSending()) {
+ Thread.sleep(1);
+ }
+ client.stopRunning();
+ rpcClient.close();
+ }
+ }
+
+
@Test (timeout = 900000)
public void testRpcWithChaosMonkeyWithSyncClient() throws Throwable {
for (int i = 0; i < numIterations; i++) {
diff --git hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index c864580..430c8a6 100644
--- hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -80,7 +80,7 @@ import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.mapreduce.TableMapper;
import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl;
import org.apache.hadoop.hbase.mapreduce.WALPlayer;
-import org.apache.hadoop.hbase.regionserver.FlushLargeStoresPolicy;
+import org.apache.hadoop.hbase.regionserver.FlushAllLargeStoresPolicy;
import org.apache.hadoop.hbase.regionserver.FlushPolicyFactory;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
@@ -1586,7 +1586,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
Configuration conf = getTestingUtil(getConf()).getConfiguration();
if (isMultiUnevenColumnFamilies(getConf())) {
// make sure per CF flush is on
- conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName());
+ conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName());
}
int ret =
ToolRunner.run(conf, new Loop(), new String[] { "1", "1", "2000000",
diff --git hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
index 2a146b3..f8a8ffa 100644
--- hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
+++ hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
@@ -41,10 +41,10 @@ import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.BufferedMutatorParams;
+import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@@ -452,7 +452,7 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
@Override
protected void handleFailure(Counters counters) throws IOException {
Configuration conf = job.getConfiguration();
- HConnection conn = (HConnection) ConnectionFactory.createConnection(conf);
+ ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
TableName tableName = TableName.valueOf(COMMON_TABLE_NAME);
CounterGroup g = counters.getGroup("undef");
Iterator it = g.iterator();
diff --git hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index 7e58420..ee61841 100644
--- hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -79,6 +79,9 @@ public abstract class Procedure implements Comparable {
private int childrenLatch = 0;
private long lastUpdate;
+ // TODO: it will be nice having pointers to allow the scheduler doing suspend/resume tricks
+ private boolean suspended = false;
+
private RemoteProcedureException exception = null;
private byte[] result = null;
@@ -94,7 +97,7 @@ public abstract class Procedure implements Comparable {
* @throws InterruptedException the procedure will be added back to the queue and retried later
*/
protected abstract Procedure[] execute(TEnvironment env)
- throws ProcedureYieldException, InterruptedException;
+ throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException;
/**
* The code to undo what done by the execute() code.
@@ -276,6 +279,9 @@ public abstract class Procedure implements Comparable {
*/
protected void toStringState(StringBuilder builder) {
builder.append(getState());
+ if (isSuspended()) {
+ builder.append("|SUSPENDED");
+ }
}
/**
@@ -319,7 +325,7 @@ public abstract class Procedure implements Comparable {
}
public long getParentProcId() {
- return parentProcId;
+ return parentProcId.longValue();
}
public NonceKey getNonceKey() {
@@ -371,6 +377,23 @@ public abstract class Procedure implements Comparable {
return false;
}
+ /**
+ * @return true if the procedure is in a suspended state,
+ * waiting for the resources required to execute the procedure will become available.
+ */
+ public synchronized boolean isSuspended() {
+ return suspended;
+ }
+
+ public synchronized void suspend() {
+ suspended = true;
+ }
+
+ public synchronized void resume() {
+ assert isSuspended() : this + " expected suspended state, got " + state;
+ suspended = false;
+ }
+
public synchronized RemoteProcedureException getException() {
return exception;
}
@@ -398,7 +421,7 @@ public abstract class Procedure implements Comparable {
* @return the timeout in msec
*/
public int getTimeout() {
- return timeout;
+ return timeout.intValue();
}
/**
@@ -494,7 +517,7 @@ public abstract class Procedure implements Comparable {
*/
@InterfaceAudience.Private
protected Procedure[] doExecute(final TEnvironment env)
- throws ProcedureYieldException, InterruptedException {
+ throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
try {
updateTimestamp();
return execute(env);
diff --git hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index f43b65f..9d71f65 100644
--- hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -505,15 +505,25 @@ public class ProcedureExecutor {
}
};
+ long st, et;
+
// Acquire the store lease.
+ st = EnvironmentEdgeManager.currentTime();
store.recoverLease();
+ et = EnvironmentEdgeManager.currentTime();
+ LOG.info(String.format("recover procedure store (%s) lease: %s",
+ store.getClass().getSimpleName(), StringUtils.humanTimeDiff(et - st)));
// TODO: Split in two steps.
// TODO: Handle corrupted procedures (currently just a warn)
// The first one will make sure that we have the latest id,
// so we can start the threads and accept new procedures.
// The second step will do the actual load of old procedures.
+ st = EnvironmentEdgeManager.currentTime();
load(abortOnCorruption);
+ et = EnvironmentEdgeManager.currentTime();
+ LOG.info(String.format("load procedure store (%s): %s",
+ store.getClass().getSimpleName(), StringUtils.humanTimeDiff(et - st)));
// Start the executors. Here we must have the lastProcId set.
for (int i = 0; i < threads.length; ++i) {
@@ -840,7 +850,7 @@ public class ProcedureExecutor {
}
// Execute the procedure
- assert proc.getState() == ProcedureState.RUNNABLE;
+ assert proc.getState() == ProcedureState.RUNNABLE : proc;
if (proc.acquireLock(getEnvironment())) {
execProcedure(procStack, proc);
proc.releaseLock(getEnvironment());
@@ -1042,6 +1052,7 @@ public class ProcedureExecutor {
Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE);
// Execute the procedure
+ boolean isSuspended = false;
boolean reExecute = false;
Procedure[] subprocs = null;
do {
@@ -1051,6 +1062,8 @@ public class ProcedureExecutor {
if (subprocs != null && subprocs.length == 0) {
subprocs = null;
}
+ } catch (ProcedureSuspendedException e) {
+ isSuspended = true;
} catch (ProcedureYieldException e) {
if (LOG.isTraceEnabled()) {
LOG.trace("Yield procedure: " + procedure + ": " + e.getMessage());
@@ -1086,7 +1099,7 @@ public class ProcedureExecutor {
break;
}
- assert subproc.getState() == ProcedureState.INITIALIZING;
+ assert subproc.getState() == ProcedureState.INITIALIZING : subproc;
subproc.setParentProcId(procedure.getProcId());
subproc.setProcId(nextProcId());
}
@@ -1107,7 +1120,7 @@ public class ProcedureExecutor {
}
} else if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) {
waitingTimeout.add(procedure);
- } else {
+ } else if (!isSuspended) {
// No subtask, so we are done
procedure.setState(ProcedureState.FINISHED);
}
diff --git hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSuspendedException.java hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSuspendedException.java
new file mode 100644
index 0000000..f28d57a
--- /dev/null
+++ hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSuspendedException.java
@@ -0,0 +1,39 @@
+/**
+ * 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.procedure2;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class ProcedureSuspendedException extends ProcedureException {
+ /** default constructor */
+ public ProcedureSuspendedException() {
+ super();
+ }
+
+ /**
+ * Constructor
+ * @param s message
+ */
+ public ProcedureSuspendedException(String s) {
+ super(s);
+ }
+}
diff --git hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
index 636a037..f0bcdea 100644
--- hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
+++ hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
@@ -42,7 +42,7 @@ public abstract class SequentialProcedure extends Procedure corrupted = new ArrayList();
+ private final ArrayList loaded = new ArrayList();
+
+ private Set procIds;
+ private long maxProcId = 0;
+
+ public LoadCounter() {
+ this(null);
+ }
+
+ public LoadCounter(final Set procIds) {
+ this.procIds = procIds;
+ }
+
+ public void reset() {
+ reset(null);
+ }
+
+ public void reset(final Set procIds) {
+ corrupted.clear();
+ loaded.clear();
+ this.procIds = procIds;
+ this.maxProcId = 0;
+ }
+
+ public long getMaxProcId() {
+ return maxProcId;
+ }
+
+ public ArrayList getLoaded() {
+ return loaded;
+ }
+
+ public int getLoadedCount() {
+ return loaded.size();
+ }
+
+ public ArrayList getCorrupted() {
+ return corrupted;
+ }
+
+ public int getCorruptedCount() {
+ return corrupted.size();
+ }
+
+ @Override
+ public void setMaxProcId(long maxProcId) {
+ maxProcId = maxProcId;
+ }
+
+ @Override
+ public void load(ProcedureIterator procIter) throws IOException {
+ while (procIter.hasNext()) {
+ Procedure proc = procIter.nextAsProcedure();
+ LOG.debug("loading procId=" + proc.getProcId() + ": " + proc);
+ if (procIds != null) {
+ assertTrue("procId=" + proc.getProcId() + " unexpected",
+ procIds.contains(proc.getProcId()));
+ }
+ loaded.add(proc);
+ }
+ }
+
+ @Override
+ public void handleCorrupted(ProcedureIterator procIter) throws IOException {
+ while (procIter.hasNext()) {
+ Procedure proc = procIter.nextAsProcedure();
+ LOG.debug("corrupted procId=" + proc.getProcId() + ": " + proc);
+ corrupted.add(proc);
+ }
+ }
+ }
}
diff --git hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java
new file mode 100644
index 0000000..1c1af79
--- /dev/null
+++ hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java
@@ -0,0 +1,134 @@
+/**
+ * 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.procedure2.store.wal;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.LoadCounter;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.IOUtils;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category({MasterTests.class, LargeTests.class})
+public class TestStressWALProcedureStore {
+ private static final Log LOG = LogFactory.getLog(TestWALProcedureStore.class);
+
+ private static final int PROCEDURE_STORE_SLOTS = 8;
+
+ private WALProcedureStore procStore;
+
+ private HBaseCommonTestingUtility htu;
+ private FileSystem fs;
+ private Path testDir;
+ private Path logDir;
+
+ private void setupConfiguration(Configuration conf) {
+ conf.setBoolean("hbase.procedure.store.wal.use.hsync", false);
+ conf.setInt("hbase.procedure.store.wal.periodic.roll.msec", 5000);
+ conf.setInt("hbase.procedure.store.wal.roll.threshold", 128 * 1024);
+ }
+
+ @Before
+ public void setUp() throws IOException {
+ htu = new HBaseCommonTestingUtility();
+ setupConfiguration(htu.getConfiguration());
+
+ testDir = htu.getDataTestDir();
+ fs = testDir.getFileSystem(htu.getConfiguration());
+ assertTrue(testDir.depth() > 1);
+
+ logDir = new Path(testDir, "proc-logs");
+ procStore = ProcedureTestingUtility.createWalStore(htu.getConfiguration(), fs, logDir);
+ procStore.start(PROCEDURE_STORE_SLOTS);
+ procStore.recoverLease();
+
+ LoadCounter loader = new LoadCounter();
+ procStore.load(loader);
+ assertEquals(0, loader.getMaxProcId());
+ assertEquals(0, loader.getLoadedCount());
+ assertEquals(0, loader.getCorruptedCount());
+ }
+
+ @After
+ public void tearDown() throws IOException {
+ procStore.stop(false);
+ fs.delete(logDir, true);
+ }
+
+ @Test
+ public void testInsertUpdateDelete() throws Exception {
+ final long LAST_PROC_ID = 19999;
+ final Thread[] thread = new Thread[PROCEDURE_STORE_SLOTS];
+ final AtomicLong procCounter = new AtomicLong((long)Math.round(Math.random() * 100));
+ for (int i = 0; i < thread.length; ++i) {
+ thread[i] = new Thread() {
+ @Override
+ public void run() {
+ Random rand = new Random();
+ TestProcedure proc;
+ do {
+ proc = new TestProcedure(procCounter.addAndGet(1));
+ // Insert
+ procStore.insert(proc, null);
+ // Update
+ for (int i = 0, nupdates = rand.nextInt(10); i <= nupdates; ++i) {
+ try { Thread.sleep(0, rand.nextInt(15)); } catch (InterruptedException e) {}
+ procStore.update(proc);
+ }
+ // Delete
+ procStore.delete(proc.getProcId());
+ } while (proc.getProcId() < LAST_PROC_ID);
+ }
+ };
+ thread[i].start();
+ }
+
+ for (int i = 0; i < thread.length; ++i) {
+ thread[i].join();
+ }
+
+ procStore.getStoreTracker().dump();
+ assertTrue(procCounter.get() >= LAST_PROC_ID);
+ assertTrue(procStore.getStoreTracker().isEmpty());
+ assertEquals(1, procStore.getActiveLogs().size());
+ }
+}
diff --git hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
index 88c85ba..f964d86 100644
--- hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
+++ hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
@@ -22,13 +22,10 @@ import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashSet;
import java.util.Set;
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -38,6 +35,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.LoadCounter;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure;
import org.apache.hadoop.hbase.procedure2.SequentialProcedure;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
@@ -441,66 +439,6 @@ public class TestWALProcedureStore {
}
@Test
- public void testInsertUpdateDelete() throws Exception {
- final int NTHREAD = 2;
-
- procStore.stop(false);
- fs.delete(logDir, true);
-
- org.apache.hadoop.conf.Configuration conf =
- new org.apache.hadoop.conf.Configuration(htu.getConfiguration());
- conf.setBoolean("hbase.procedure.store.wal.use.hsync", false);
- conf.setInt("hbase.procedure.store.wal.periodic.roll.msec", 10000);
- conf.setInt("hbase.procedure.store.wal.roll.threshold", 128 * 1024);
-
- fs.mkdirs(logDir);
- procStore = ProcedureTestingUtility.createWalStore(conf, fs, logDir);
- procStore.start(NTHREAD);
- procStore.recoverLease();
-
- LoadCounter loader = new LoadCounter();
- procStore.load(loader);
- assertEquals(0, loader.getMaxProcId());
- assertEquals(0, loader.getLoadedCount());
- assertEquals(0, loader.getCorruptedCount());
-
- final long LAST_PROC_ID = 9999;
- final Thread[] thread = new Thread[NTHREAD];
- final AtomicLong procCounter = new AtomicLong((long)Math.round(Math.random() * 100));
- for (int i = 0; i < thread.length; ++i) {
- thread[i] = new Thread() {
- @Override
- public void run() {
- Random rand = new Random();
- TestProcedure proc;
- do {
- proc = new TestProcedure(procCounter.addAndGet(1));
- // Insert
- procStore.insert(proc, null);
- // Update
- for (int i = 0, nupdates = rand.nextInt(10); i <= nupdates; ++i) {
- try { Thread.sleep(0, rand.nextInt(15)); } catch (InterruptedException e) {}
- procStore.update(proc);
- }
- // Delete
- procStore.delete(proc.getProcId());
- } while (proc.getProcId() < LAST_PROC_ID);
- }
- };
- thread[i].start();
- }
-
- for (int i = 0; i < thread.length; ++i) {
- thread[i].join();
- }
-
- procStore.getStoreTracker().dump();
- assertTrue(procCounter.get() >= LAST_PROC_ID);
- assertTrue(procStore.getStoreTracker().isEmpty());
- assertEquals(1, procStore.getActiveLogs().size());
- }
-
- @Test
public void testRollAndRemove() throws IOException {
// Insert something in the log
Procedure proc1 = new TestSequentialProcedure();
@@ -599,78 +537,4 @@ public class TestWALProcedureStore {
}
}
}
-
- private class LoadCounter implements ProcedureStore.ProcedureLoader {
- private final ArrayList corrupted = new ArrayList();
- private final ArrayList loaded = new ArrayList();
-
- private Set procIds;
- private long maxProcId = 0;
-
- public LoadCounter() {
- this(null);
- }
-
- public LoadCounter(final Set procIds) {
- this.procIds = procIds;
- }
-
- public void reset() {
- reset(null);
- }
-
- public void reset(final Set procIds) {
- corrupted.clear();
- loaded.clear();
- this.procIds = procIds;
- this.maxProcId = 0;
- }
-
- public long getMaxProcId() {
- return maxProcId;
- }
-
- public ArrayList getLoaded() {
- return loaded;
- }
-
- public int getLoadedCount() {
- return loaded.size();
- }
-
- public ArrayList getCorrupted() {
- return corrupted;
- }
-
- public int getCorruptedCount() {
- return corrupted.size();
- }
-
- @Override
- public void setMaxProcId(long maxProcId) {
- maxProcId = maxProcId;
- }
-
- @Override
- public void load(ProcedureIterator procIter) throws IOException {
- while (procIter.hasNext()) {
- Procedure proc = procIter.nextAsProcedure();
- LOG.debug("loading procId=" + proc.getProcId() + ": " + proc);
- if (procIds != null) {
- assertTrue("procId=" + proc.getProcId() + " unexpected",
- procIds.contains(proc.getProcId()));
- }
- loaded.add(proc);
- }
- }
-
- @Override
- public void handleCorrupted(ProcedureIterator procIter) throws IOException {
- while (procIter.hasNext()) {
- Procedure proc = procIter.nextAsProcedure();
- LOG.debug("corrupted procId=" + proc.getProcId() + ": " + proc);
- corrupted.add(proc);
- }
- }
- }
}
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
index c88ac91..8ff3ef6 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
@@ -86,7 +86,9 @@ public class MultiRowResource extends ResourceBase implements Constants {
}
model.addRow(rowModel);
} else {
- LOG.trace("The row : " + rk + " not found in the table.");
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("The row : " + rk + " not found in the table.");
+ }
}
}
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
index 8f64738..c832905 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
@@ -91,8 +91,8 @@ public class NamespacesInstanceResource extends ResourceBase {
MIMETYPE_PROTOBUF_IETF})
public Response get(final @Context ServletContext context,
final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
@@ -135,8 +135,8 @@ public class NamespacesInstanceResource extends ResourceBase {
@Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
public Response put(final NamespacesInstanceModel model, final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("PUT " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("PUT " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
return processUpdate(model, true, uriInfo);
@@ -151,8 +151,8 @@ public class NamespacesInstanceResource extends ResourceBase {
@PUT
public Response putNoBody(final byte[] message,
final @Context UriInfo uriInfo, final @Context HttpHeaders headers) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("PUT " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("PUT " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
try{
@@ -176,8 +176,8 @@ public class NamespacesInstanceResource extends ResourceBase {
public Response post(final NamespacesInstanceModel model,
final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("POST " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("POST " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
return processUpdate(model, false, uriInfo);
@@ -192,8 +192,8 @@ public class NamespacesInstanceResource extends ResourceBase {
@POST
public Response postNoBody(final byte[] message,
final @Context UriInfo uriInfo, final @Context HttpHeaders headers) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("POST " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("POST " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
try{
@@ -287,8 +287,8 @@ public class NamespacesInstanceResource extends ResourceBase {
@DELETE
public Response deleteNoBody(final byte[] message,
final @Context UriInfo uriInfo, final @Context HttpHeaders headers) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("DELETE " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("DELETE " + uriInfo.getAbsolutePath());
}
if (servlet.isReadOnly()) {
servlet.getMetrics().incrementFailedDeleteRequests(1);
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesResource.java
index 0548fe8..1304fe0 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesResource.java
@@ -64,8 +64,8 @@ public class NamespacesResource extends ResourceBase {
@Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
public Response get(final @Context ServletContext context, final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
try {
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ProtobufStreamingUtil.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ProtobufStreamingUtil.java
index 93bb940..cb0f4c8 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ProtobufStreamingUtil.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ProtobufStreamingUtil.java
@@ -49,8 +49,10 @@ public class ProtobufStreamingUtil implements StreamingOutput {
this.contentType = type;
this.limit = limit;
this.fetchSize = fetchSize;
- LOG.debug("Created ScanStreamingUtil with content type = " + this.contentType + " user limit : "
- + this.limit + " scan fetch size : " + this.fetchSize);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Created ScanStreamingUtil with content type = " + this.contentType
+ + " user limit : " + this.limit + " scan fetch size : " + this.fetchSize);
+ }
}
@Override
@@ -82,7 +84,9 @@ public class ProtobufStreamingUtil implements StreamingOutput {
outStream.write(Bytes.toBytes((short)objectBytes.length));
outStream.write(objectBytes);
outStream.flush();
- LOG.trace("Wrote " + model.getRows().size() + " rows to stream successfully.");
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Wrote " + model.getRows().size() + " rows to stream successfully.");
+ }
}
private CellSetModel createModelFromResults(Result[] results) {
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
index b26de54..cb37fb5 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
@@ -168,20 +168,26 @@ public class RESTServer implements Constants {
if (commandLine != null && commandLine.hasOption("port")) {
String val = commandLine.getOptionValue("port");
servlet.getConfiguration().setInt("hbase.rest.port", Integer.parseInt(val));
- LOG.debug("port set to " + val);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("port set to " + val);
+ }
}
// check if server should only process GET requests, if so override the conf
if (commandLine != null && commandLine.hasOption("readonly")) {
servlet.getConfiguration().setBoolean("hbase.rest.readonly", true);
- LOG.debug("readonly set to true");
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("readonly set to true");
+ }
}
// check for user-defined info server port setting, if so override the conf
if (commandLine != null && commandLine.hasOption("infoport")) {
String val = commandLine.getOptionValue("infoport");
servlet.getConfiguration().setInt("hbase.rest.info.port", Integer.parseInt(val));
- LOG.debug("Web UI port set to " + val);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Web UI port set to " + val);
+ }
}
@SuppressWarnings("unchecked")
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java
index 4da5c67..411ced8 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.rest;
import java.io.IOException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
@@ -30,14 +32,13 @@ import org.apache.hadoop.hbase.util.ConnectionCache;
import org.apache.hadoop.hbase.util.JvmPauseMonitor;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.log4j.Logger;
/**
* Singleton class encapsulating global REST servlet state and functions.
*/
@InterfaceAudience.Private
public class RESTServlet implements Constants {
- private static final Logger LOG = Logger.getLogger(RESTServlet.class);
+ private static final Log LOG = LogFactory.getLog(RESTServlet.class);
private static RESTServlet INSTANCE;
private final Configuration conf;
private final MetricsREST metrics;
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
index 48721bb..f803b26 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
@@ -71,8 +71,8 @@ public class RegionsResource extends ResourceBase {
@Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
public Response get(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
try {
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
index c08bb8b..fc4c548 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
@@ -72,8 +72,8 @@ public class RootResource extends ResourceBase {
@Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
public Response get(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
try {
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
index 15828ce..de84625 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
@@ -85,8 +85,8 @@ public class RowResource extends ResourceBase {
@Produces({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
public Response get(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
MultivaluedMap params = uriInfo.getQueryParameters();
@@ -130,8 +130,8 @@ public class RowResource extends ResourceBase {
@GET
@Produces(MIMETYPE_BINARY)
public Response getBinary(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath() + " as "+ MIMETYPE_BINARY);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath() + " as "+ MIMETYPE_BINARY);
}
servlet.getMetrics().incrementRequests(1);
// doesn't make sense to use a non specific coordinate as this can only
@@ -221,8 +221,8 @@ public class RowResource extends ResourceBase {
put.addImmutable(parts[0], parts[1], cell.getTimestamp(), cell.getValue());
}
puts.add(put);
- if (LOG.isDebugEnabled()) {
- LOG.debug("PUT " + put.toString());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("PUT " + put.toString());
}
}
table = servlet.getTable(tableResource.getName());
@@ -289,8 +289,8 @@ public class RowResource extends ResourceBase {
put.addImmutable(parts[0], parts[1], timestamp, message);
table = servlet.getTable(tableResource.getName());
table.put(put);
- if (LOG.isDebugEnabled()) {
- LOG.debug("PUT " + put.toString());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("PUT " + put.toString());
}
servlet.getMetrics().incrementSucessfulPutRequests(1);
return Response.ok().build();
@@ -301,7 +301,7 @@ public class RowResource extends ResourceBase {
if (table != null) try {
table.close();
} catch (IOException ioe) {
- LOG.debug(ioe);
+ LOG.debug("Exception received while closing the table", ioe);
}
}
}
@@ -311,8 +311,8 @@ public class RowResource extends ResourceBase {
MIMETYPE_PROTOBUF_IETF})
public Response put(final CellSetModel model,
final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("PUT " + uriInfo.getAbsolutePath()
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("PUT " + uriInfo.getAbsolutePath()
+ " " + uriInfo.getQueryParameters());
}
return update(model, true);
@@ -322,8 +322,8 @@ public class RowResource extends ResourceBase {
@Consumes(MIMETYPE_BINARY)
public Response putBinary(final byte[] message,
final @Context UriInfo uriInfo, final @Context HttpHeaders headers) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("PUT " + uriInfo.getAbsolutePath() + " as "+ MIMETYPE_BINARY);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("PUT " + uriInfo.getAbsolutePath() + " as "+ MIMETYPE_BINARY);
}
return updateBinary(message, headers, true);
}
@@ -333,8 +333,8 @@ public class RowResource extends ResourceBase {
MIMETYPE_PROTOBUF_IETF})
public Response post(final CellSetModel model,
final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("POST " + uriInfo.getAbsolutePath()
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("POST " + uriInfo.getAbsolutePath()
+ " " + uriInfo.getQueryParameters());
}
return update(model, false);
@@ -344,16 +344,16 @@ public class RowResource extends ResourceBase {
@Consumes(MIMETYPE_BINARY)
public Response postBinary(final byte[] message,
final @Context UriInfo uriInfo, final @Context HttpHeaders headers) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("POST " + uriInfo.getAbsolutePath() + " as "+MIMETYPE_BINARY);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("POST " + uriInfo.getAbsolutePath() + " as "+MIMETYPE_BINARY);
}
return updateBinary(message, headers, false);
}
@DELETE
public Response delete(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("DELETE " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("DELETE " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
if (servlet.isReadOnly()) {
@@ -397,8 +397,8 @@ public class RowResource extends ResourceBase {
table = servlet.getTable(tableResource.getName());
table.delete(delete);
servlet.getMetrics().incrementSucessfulDeleteRequests(1);
- if (LOG.isDebugEnabled()) {
- LOG.debug("DELETE " + delete.toString());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("DELETE " + delete.toString());
}
} catch (Exception e) {
servlet.getMetrics().incrementFailedDeleteRequests(1);
@@ -407,7 +407,7 @@ public class RowResource extends ResourceBase {
if (table != null) try {
table.close();
} catch (IOException ioe) {
- LOG.debug(ioe);
+ LOG.debug("Exception received while closing the table", ioe);
}
}
return Response.ok().build();
@@ -499,8 +499,8 @@ public class RowResource extends ResourceBase {
.build();
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("CHECK-AND-PUT " + put.toString() + ", returns " + retValue);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("CHECK-AND-PUT " + put.toString() + ", returns " + retValue);
}
if (!retValue) {
servlet.getMetrics().incrementFailedPutRequests(1);
@@ -517,7 +517,7 @@ public class RowResource extends ResourceBase {
} finally {
if (table != null) try {
table.close();
- } catch (IOException ioe) {
+ } catch (IOException ioe) {
LOG.debug("Exception received while closing the table", ioe);
}
}
@@ -627,8 +627,8 @@ public class RowResource extends ResourceBase {
.build();
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("CHECK-AND-DELETE " + delete.toString() + ", returns "
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("CHECK-AND-DELETE " + delete.toString() + ", returns "
+ retValue);
}
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java
index ffb2fae..2469faa 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java
@@ -62,7 +62,7 @@ public class ScannerInstanceResource extends ResourceBase {
public ScannerInstanceResource() throws IOException { }
- public ScannerInstanceResource(String table, String id,
+ public ScannerInstanceResource(String table, String id,
ResultGenerator generator, int batch) throws IOException {
this.id = id;
this.generator = generator;
@@ -72,10 +72,10 @@ public class ScannerInstanceResource extends ResourceBase {
@GET
@Produces({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
- public Response get(final @Context UriInfo uriInfo,
+ public Response get(final @Context UriInfo uriInfo,
@QueryParam("n") int maxRows, final @QueryParam("c") int maxValues) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
if (generator == null) {
@@ -108,7 +108,9 @@ public class ScannerInstanceResource extends ResourceBase {
.build();
}
if (value == null) {
- LOG.info("generator exhausted");
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("generator exhausted");
+ }
// respond with 204 (No Content) if an empty cell set would be
// returned
if (count == limit) {
@@ -123,7 +125,7 @@ public class ScannerInstanceResource extends ResourceBase {
if (!Bytes.equals(CellUtil.cloneRow(value), rowKey)) {
// if maxRows was given as a query param, stop if we would exceed the
// specified number of rows
- if (maxRows > 0) {
+ if (maxRows > 0) {
if (--maxRows == 0) {
generator.putBack(value);
break;
@@ -134,7 +136,7 @@ public class ScannerInstanceResource extends ResourceBase {
rowModel = new RowModel(rowKey);
}
rowModel.addCell(
- new CellModel(CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value),
+ new CellModel(CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value),
value.getTimestamp(), CellUtil.cloneValue(value)));
} while (--count > 0);
model.addRow(rowModel);
@@ -147,21 +149,23 @@ public class ScannerInstanceResource extends ResourceBase {
@GET
@Produces(MIMETYPE_BINARY)
public Response getBinary(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath() + " as " +
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath() + " as " +
MIMETYPE_BINARY);
}
servlet.getMetrics().incrementRequests(1);
try {
Cell value = generator.next();
if (value == null) {
- LOG.info("generator exhausted");
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("generator exhausted");
+ }
return Response.noContent().build();
}
ResponseBuilder response = Response.ok(CellUtil.cloneValue(value));
response.cacheControl(cacheControl);
- response.header("X-Row", Base64.encodeBytes(CellUtil.cloneRow(value)));
- response.header("X-Column",
+ response.header("X-Row", Base64.encodeBytes(CellUtil.cloneRow(value)));
+ response.header("X-Column",
Base64.encodeBytes(
KeyValue.makeColumn(CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value))));
response.header("X-Timestamp", value.getTimestamp());
@@ -182,8 +186,8 @@ public class ScannerInstanceResource extends ResourceBase {
@DELETE
public Response delete(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("DELETE " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("DELETE " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
if (servlet.isReadOnly()) {
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerResource.java
index 6c424ce..71723d8 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/ScannerResource.java
@@ -31,7 +31,6 @@ import javax.ws.rs.PUT;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MultivaluedMap;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.UriBuilder;
import javax.ws.rs.core.UriInfo;
@@ -91,8 +90,7 @@ public class ScannerResource extends ResourceBase {
spec = new RowSpec(model.getStartRow(), endRow, model.getColumns(), model.getStartTime(),
model.getEndTime(), model.getMaxVersions());
}
- MultivaluedMap params = uriInfo.getQueryParameters();
-
+
try {
Filter filter = ScannerResultGenerator.buildFilterFromModel(model);
String tableName = tableResource.getName();
@@ -103,8 +101,8 @@ public class ScannerResource extends ResourceBase {
ScannerInstanceResource instance =
new ScannerInstanceResource(tableName, id, gen, model.getBatch());
scanners.put(id, instance);
- if (LOG.isDebugEnabled()) {
- LOG.debug("new scanner: " + id);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("new scanner: " + id);
}
UriBuilder builder = uriInfo.getAbsolutePathBuilder();
URI uri = builder.path(id).build();
@@ -130,10 +128,10 @@ public class ScannerResource extends ResourceBase {
@PUT
@Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
- public Response put(final ScannerModel model,
+ public Response put(final ScannerModel model,
final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("PUT " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("PUT " + uriInfo.getAbsolutePath());
}
return update(model, true, uriInfo);
}
@@ -143,8 +141,8 @@ public class ScannerResource extends ResourceBase {
MIMETYPE_PROTOBUF_IETF})
public Response post(final ScannerModel model,
final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("POST " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("POST " + uriInfo.getAbsolutePath());
}
return update(model, false, uriInfo);
}
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
index c0e7153..dc34f09 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
@@ -86,8 +86,8 @@ public class SchemaResource extends ResourceBase {
@Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
public Response get(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
try {
@@ -99,7 +99,7 @@ public class SchemaResource extends ResourceBase {
} catch (Exception e) {
servlet.getMetrics().incrementFailedGetRequests(1);
return processException(e);
- }
+ }
}
private Response replace(final TableName name, final TableSchemaModel model,
@@ -198,10 +198,10 @@ public class SchemaResource extends ResourceBase {
@PUT
@Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
- public Response put(final TableSchemaModel model,
+ public Response put(final TableSchemaModel model,
final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("PUT " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("PUT " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
return update(model, true, uriInfo);
@@ -210,10 +210,10 @@ public class SchemaResource extends ResourceBase {
@POST
@Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
- public Response post(final TableSchemaModel model,
+ public Response post(final TableSchemaModel model,
final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("PUT " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("PUT " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
return update(model, false, uriInfo);
@@ -223,8 +223,8 @@ public class SchemaResource extends ResourceBase {
justification="Expected")
@DELETE
public Response delete(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("DELETE " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("DELETE " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
if (servlet.isReadOnly()) {
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
index a7e52bd..27977c3 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
@@ -63,8 +63,8 @@ public class StorageClusterStatusResource extends ResourceBase {
@Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
public Response get(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
try {
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
index 85e81f8..b9fb5d4 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
@@ -58,8 +58,8 @@ public class StorageClusterVersionResource extends ResourceBase {
@GET
@Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON})
public Response get(final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
try {
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java
index f87ef7e..70a4538 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java
@@ -133,7 +133,7 @@ public class TableResource extends ResourceBase {
@DefaultValue("-1") @QueryParam(Constants.SCAN_BATCH_SIZE) int batchSize,
@DefaultValue("0") @QueryParam(Constants.SCAN_START_TIME) long startTime,
@DefaultValue(Long.MAX_VALUE + "") @QueryParam(Constants.SCAN_END_TIME) long endTime,
- @DefaultValue("true") @QueryParam(Constants.SCAN_BATCH_SIZE) boolean cacheBlocks,
+ @DefaultValue("true") @QueryParam(Constants.SCAN_BATCH_SIZE) boolean cacheBlocks,
@DefaultValue("") @QueryParam(Constants.SCAN_FILTER) String filters) {
try {
Filter filter = null;
@@ -146,10 +146,12 @@ public class TableResource extends ResourceBase {
tableScan.setStartRow(prefixBytes);
}
}
- LOG.debug("Query parameters : Table Name = > " + this.table + " Start Row => " + startRow
- + " End Row => " + endRow + " Columns => " + column + " Start Time => " + startTime
- + " End Time => " + endTime + " Cache Blocks => " + cacheBlocks + " Max Versions => "
- + maxVersions + " Batch Size => " + batchSize);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Query parameters : Table Name = > " + this.table + " Start Row => " + startRow
+ + " End Row => " + endRow + " Columns => " + column + " Start Time => " + startTime
+ + " End Time => " + endTime + " Cache Blocks => " + cacheBlocks + " Max Versions => "
+ + maxVersions + " Batch Size => " + batchSize);
+ }
Table hTable = RESTServlet.getInstance().getTable(this.table);
tableScan.setBatch(batchSize);
tableScan.setMaxVersions(maxVersions);
@@ -162,15 +164,21 @@ public class TableResource extends ResourceBase {
String[] familysplit = csplit.trim().split(":");
if (familysplit.length == 2) {
if (familysplit[1].length() > 0) {
- LOG.debug("Scan family and column : " + familysplit[0] + " " + familysplit[1]);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Scan family and column : " + familysplit[0] + " " + familysplit[1]);
+ }
tableScan.addColumn(Bytes.toBytes(familysplit[0]), Bytes.toBytes(familysplit[1]));
} else {
tableScan.addFamily(Bytes.toBytes(familysplit[0]));
- LOG.debug("Scan family : " + familysplit[0] + " and empty qualifier.");
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Scan family : " + familysplit[0] + " and empty qualifier.");
+ }
tableScan.addColumn(Bytes.toBytes(familysplit[0]), null);
}
- } else if (StringUtils.isNotEmpty(familysplit[0])){
- LOG.debug("Scan family : " + familysplit[0]);
+ } else if (StringUtils.isNotEmpty(familysplit[0])) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Scan family : " + familysplit[0]);
+ }
tableScan.addFamily(Bytes.toBytes(familysplit[0]));
}
}
@@ -192,8 +200,8 @@ public class TableResource extends ResourceBase {
}
int fetchSize = this.servlet.getConfiguration().getInt(Constants.SCAN_FETCH_SIZE, 10);
tableScan.setCaching(fetchSize);
- return new TableScanResource(hTable.getScanner(tableScan), userRequestedLimit);
- } catch (Exception exp) {
+ return new TableScanResource(hTable.getScanner(tableScan), userRequestedLimit);
+ } catch (IOException exp) {
servlet.getMetrics().incrementFailedScanRequests(1);
processException(exp);
LOG.warn(exp);
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/VersionResource.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/VersionResource.java
index ae93825..172246c 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/VersionResource.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/VersionResource.java
@@ -68,15 +68,15 @@ public class VersionResource extends ResourceBase {
* Build a response for a version request.
* @param context servlet context
* @param uriInfo (JAX-RS context variable) request URL
- * @return a response for a version request
+ * @return a response for a version request
*/
@GET
@Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
MIMETYPE_PROTOBUF_IETF})
- public Response get(final @Context ServletContext context,
+ public Response get(final @Context ServletContext context,
final @Context UriInfo uriInfo) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("GET " + uriInfo.getAbsolutePath());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("GET " + uriInfo.getAbsolutePath());
}
servlet.getMetrics().incrementRequests(1);
ResponseBuilder response = Response.ok(new VersionModel(context));
@@ -89,7 +89,7 @@ public class VersionResource extends ResourceBase {
* Dispatch to StorageClusterVersionResource
*/
@Path("cluster")
- public StorageClusterVersionResource getClusterVersionResource()
+ public StorageClusterVersionResource getClusterVersionResource()
throws IOException {
return new StorageClusterVersionResource();
}
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
index 142c276..e26de63 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
@@ -101,10 +101,10 @@ public class Client {
}
/**
- * Shut down the client. Close any open persistent connections.
+ * Shut down the client. Close any open persistent connections.
*/
public void shutdown() {
- MultiThreadedHttpConnectionManager manager =
+ MultiThreadedHttpConnectionManager manager =
(MultiThreadedHttpConnectionManager) httpClient.getHttpConnectionManager();
manager.shutdown();
}
@@ -151,7 +151,7 @@ public class Client {
* one of the members of the supplied cluster definition and iterate through
* the list until a transaction can be successfully completed. The
* definition of success here is a complete HTTP transaction, irrespective
- * of result code.
+ * of result code.
* @param cluster the cluster definition
* @param method the transaction method
* @param headers HTTP header values to send
@@ -209,8 +209,8 @@ public class Client {
long startTime = System.currentTimeMillis();
int code = httpClient.executeMethod(method);
long endTime = System.currentTimeMillis();
- if (LOG.isDebugEnabled()) {
- LOG.debug(method.getName() + " " + uri + " " + code + " " +
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(method.getName() + " " + uri + " " + code + " " +
method.getStatusText() + " in " + (endTime - startTime) + " ms");
}
return code;
@@ -250,7 +250,7 @@ public class Client {
}
/**
- * Send a HEAD request
+ * Send a HEAD request
* @param path the path or URI
* @return a Response object with response detail
* @throws IOException
@@ -260,14 +260,14 @@ public class Client {
}
/**
- * Send a HEAD request
+ * Send a HEAD request
* @param cluster the cluster definition
* @param path the path or URI
* @param headers the HTTP headers to include in the request
* @return a Response object with response detail
* @throws IOException
*/
- public Response head(Cluster cluster, String path, Header[] headers)
+ public Response head(Cluster cluster, String path, Header[] headers)
throws IOException {
HeadMethod method = new HeadMethod();
try {
@@ -280,7 +280,7 @@ public class Client {
}
/**
- * Send a GET request
+ * Send a GET request
* @param path the path or URI
* @return a Response object with response detail
* @throws IOException
@@ -290,7 +290,7 @@ public class Client {
}
/**
- * Send a GET request
+ * Send a GET request
* @param cluster the cluster definition
* @param path the path or URI
* @return a Response object with response detail
@@ -301,7 +301,7 @@ public class Client {
}
/**
- * Send a GET request
+ * Send a GET request
* @param path the path or URI
* @param accept Accept header value
* @return a Response object with response detail
@@ -312,7 +312,7 @@ public class Client {
}
/**
- * Send a GET request
+ * Send a GET request
* @param cluster the cluster definition
* @param path the path or URI
* @param accept Accept header value
@@ -329,7 +329,7 @@ public class Client {
/**
* Send a GET request
* @param path the path or URI
- * @param headers the HTTP headers to include in the request,
+ * @param headers the HTTP headers to include in the request,
* Accept must be supplied
* @return a Response object with response detail
* @throws IOException
@@ -346,7 +346,7 @@ public class Client {
* @return a Response object with response detail
* @throws IOException
*/
- public Response get(Cluster c, String path, Header[] headers)
+ public Response get(Cluster c, String path, Header[] headers)
throws IOException {
GetMethod method = new GetMethod();
try {
@@ -396,7 +396,7 @@ public class Client {
* @return a Response object with response detail
* @throws IOException for error
*/
- public Response put(Cluster cluster, String path, String contentType,
+ public Response put(Cluster cluster, String path, String contentType,
byte[] content) throws IOException {
Header[] headers = new Header[1];
headers[0] = new Header("Content-Type", contentType);
@@ -413,7 +413,7 @@ public class Client {
* @return a Response object with response detail
* @throws IOException for error
*/
- public Response put(Cluster cluster, String path, String contentType,
+ public Response put(Cluster cluster, String path, String contentType,
byte[] content, Header extraHdr) throws IOException {
int cnt = extraHdr == null ? 1 : 2;
Header[] headers = new Header[cnt];
@@ -433,7 +433,7 @@ public class Client {
* @return a Response object with response detail
* @throws IOException
*/
- public Response put(String path, Header[] headers, byte[] content)
+ public Response put(String path, Header[] headers, byte[] content)
throws IOException {
return put(cluster, path, headers, content);
}
@@ -448,7 +448,7 @@ public class Client {
* @return a Response object with response detail
* @throws IOException
*/
- public Response put(Cluster cluster, String path, Header[] headers,
+ public Response put(Cluster cluster, String path, Header[] headers,
byte[] content) throws IOException {
PutMethod method = new PutMethod();
try {
@@ -498,7 +498,7 @@ public class Client {
* @return a Response object with response detail
* @throws IOException for error
*/
- public Response post(Cluster cluster, String path, String contentType,
+ public Response post(Cluster cluster, String path, String contentType,
byte[] content) throws IOException {
Header[] headers = new Header[1];
headers[0] = new Header("Content-Type", contentType);
@@ -515,7 +515,7 @@ public class Client {
* @return a Response object with response detail
* @throws IOException for error
*/
- public Response post(Cluster cluster, String path, String contentType,
+ public Response post(Cluster cluster, String path, String contentType,
byte[] content, Header extraHdr) throws IOException {
int cnt = extraHdr == null ? 1 : 2;
Header[] headers = new Header[cnt];
@@ -535,7 +535,7 @@ public class Client {
* @return a Response object with response detail
* @throws IOException
*/
- public Response post(String path, Header[] headers, byte[] content)
+ public Response post(String path, Header[] headers, byte[] content)
throws IOException {
return post(cluster, path, headers, content);
}
@@ -550,7 +550,7 @@ public class Client {
* @return a Response object with response detail
* @throws IOException
*/
- public Response post(Cluster cluster, String path, Header[] headers,
+ public Response post(Cluster cluster, String path, Header[] headers,
byte[] content) throws IOException {
PostMethod method = new PostMethod();
try {
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/AuthFilter.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/AuthFilter.java
index e5208af..f051bc8 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/AuthFilter.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/AuthFilter.java
@@ -72,7 +72,9 @@ public class AuthFilter extends AuthenticationFilter {
throw new ServletException("Failed to retrieve server principal", ie);
}
}
- LOG.debug("Setting property " + name + "=" + value);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Setting property " + name + "=" + value);
+ }
name = name.substring(REST_PREFIX_LEN);
props.setProperty(name, value);
}
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GzipFilter.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GzipFilter.java
index 4995b86..094ae0b 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GzipFilter.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GzipFilter.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.rest.filter;
import java.io.IOException;
import java.io.OutputStream;
import java.util.HashSet;
+import java.util.Locale;
import java.util.Set;
import java.util.StringTokenizer;
@@ -65,11 +66,11 @@ public class GzipFilter implements Filter {
String acceptEncoding = request.getHeader("accept-encoding");
String contentType = request.getHeader("content-type");
if ((contentEncoding != null) &&
- (contentEncoding.toLowerCase().indexOf("gzip") > -1)) {
+ (contentEncoding.toLowerCase(Locale.ROOT).indexOf("gzip") > -1)) {
request = new GZIPRequestWrapper(request);
}
if (((acceptEncoding != null) &&
- (acceptEncoding.toLowerCase().indexOf("gzip") > -1)) ||
+ (acceptEncoding.toLowerCase(Locale.ROOT).indexOf("gzip") > -1)) ||
((contentType != null) && mimeTypes.contains(contentType))) {
response = new GZIPResponseWrapper(response);
}
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/RestCsrfPreventionFilter.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/RestCsrfPreventionFilter.java
index 30eea95..dbb1447 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/RestCsrfPreventionFilter.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/RestCsrfPreventionFilter.java
@@ -34,13 +34,12 @@ import javax.servlet.ServletResponse;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
/**
* This filter provides protection against cross site request forgery (CSRF)
* attacks for REST APIs. Enabling this filter on an endpoint results in the
@@ -52,8 +51,8 @@ import org.slf4j.LoggerFactory;
@InterfaceStability.Evolving
public class RestCsrfPreventionFilter implements Filter {
- private static final Logger LOG =
- LoggerFactory.getLogger(RestCsrfPreventionFilter.class);
+ private static final Log LOG =
+ LogFactory.getLog(RestCsrfPreventionFilter.class);
public static final String HEADER_USER_AGENT = "User-Agent";
public static final String BROWSER_USER_AGENT_PARAM =
@@ -87,9 +86,9 @@ public class RestCsrfPreventionFilter implements Filter {
agents = BROWSER_USER_AGENTS_DEFAULT;
}
parseBrowserUserAgents(agents);
- LOG.info("Adding cross-site request forgery (CSRF) protection, "
- + "headerName = {}, methodsToIgnore = {}, browserUserAgents = {}",
- headerName, methodsToIgnore, browserUserAgents);
+ LOG.info(String.format("Adding cross-site request forgery (CSRF) protection, "
+ + "headerName = %s, methodsToIgnore = %s, browserUserAgents = %s",
+ headerName, methodsToIgnore, browserUserAgents));
}
void parseBrowserUserAgents(String userAgents) {
diff --git hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/consumer/ProtobufMessageBodyConsumer.java hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/consumer/ProtobufMessageBodyConsumer.java
index ec39db0..073c038 100644
--- hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/consumer/ProtobufMessageBodyConsumer.java
+++ hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/consumer/ProtobufMessageBodyConsumer.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
@Provider
@Consumes({Constants.MIMETYPE_PROTOBUF, Constants.MIMETYPE_PROTOBUF_IETF})
@InterfaceAudience.Private
-public class ProtobufMessageBodyConsumer
+public class ProtobufMessageBodyConsumer
implements MessageBodyReader {
private static final Log LOG =
LogFactory.getLog(ProtobufMessageBodyConsumer.class);
@@ -73,8 +73,8 @@ public class ProtobufMessageBodyConsumer
baos.write(buffer, 0, read);
}
} while (read > 0);
- if (LOG.isDebugEnabled()) {
- LOG.debug(getClass() + ": read " + baos.size() + " bytes from " +
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(getClass() + ": read " + baos.size() + " bytes from " +
inputStream);
}
obj = obj.getObjectFromMessage(baos.toByteArray());
diff --git hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
index 8725781..7aea464 100644
--- hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
+++ hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
@@ -191,7 +191,7 @@ public class RSGroupAdminServer extends RSGroupAdmin {
}
}
for (RegionState state :
- master.getAssignmentManager().getRegionStates().getRegionsInTransition().values()) {
+ master.getAssignmentManager().getRegionStates().getRegionsInTransition()) {
if (state.getServerName().getHostPort().equals(rs)) {
regions.add(state.getRegion());
}
diff --git hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
index 34add63..25c54b2 100644
--- hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
+++ hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
@@ -19,7 +19,6 @@
*/
package org.apache.hadoop.hbase.rsgroup;
-import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.common.net.HostAndPort;
@@ -35,9 +34,9 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
@@ -50,20 +49,11 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
import java.io.IOException;
-import java.lang.management.ManagementFactory;
import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicReference;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -190,7 +180,8 @@ public class TestRSGroups extends TestRSGroupsBase {
});
ServerName targetServer =
ServerName.parseServerName(appInfo.getServers().iterator().next().toString());
- AdminProtos.AdminService.BlockingInterface rs = admin.getConnection().getAdmin(targetServer);
+ AdminProtos.AdminService.BlockingInterface rs =
+ ((ClusterConnection) admin.getConnection()).getAdmin(targetServer);
//verify it was assigned to the right group
Assert.assertEquals(1, ProtobufUtil.getOnlineRegions(rs).size());
}
diff --git hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
index 9225e09..b1c7b3b 100644
--- hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
+++ hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -482,7 +483,7 @@ public abstract class TestRSGroupsBase {
}
final AdminProtos.AdminService.BlockingInterface targetRS =
- admin.getConnection().getAdmin(targetServer);
+ ((ClusterConnection) admin.getConnection()).getAdmin(targetServer);
//move target server to group
rsGroupAdmin.moveServers(Sets.newHashSet(targetServer.getHostPort()),
@@ -571,7 +572,7 @@ public abstract class TestRSGroupsBase {
ServerName targetServer = ServerName.parseServerName(
appInfo.getServers().iterator().next().toString());
AdminProtos.AdminService.BlockingInterface targetRS =
- admin.getConnection().getAdmin(targetServer);
+ ((ClusterConnection) admin.getConnection()).getAdmin(targetServer);
HRegionInfo targetRegion = ProtobufUtil.getOnlineRegions(targetRS).get(0);
Assert.assertEquals(1, ProtobufUtil.getOnlineRegions(targetRS).size());
@@ -612,7 +613,7 @@ public abstract class TestRSGroupsBase {
targetServer = ServerName.parseServerName(
newServers.iterator().next().toString());
targetRS =
- admin.getConnection().getAdmin(targetServer);
+ ((ClusterConnection) admin.getConnection()).getAdmin(targetServer);
Assert.assertEquals(1, ProtobufUtil.getOnlineRegions(targetRS).size());
Assert.assertEquals(tableName,
ProtobufUtil.getOnlineRegions(targetRS).get(0).getTable());
diff --git hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
index 42334ff..e2ae09d 100644
--- hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
+++ hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
@@ -23,38 +23,34 @@ org.apache.hadoop.hbase.master.RegionState;
org.apache.hadoop.conf.Configuration;
org.apache.hadoop.hbase.HBaseConfiguration;
org.apache.hadoop.hbase.HConstants;
-java.util.Iterator;
-java.util.Map;
-java.util.List;
-java.util.ArrayList;
-java.util.Map.Entry;
-java.util.Arrays;
+java.util.HashSet;
+java.util.SortedSet;
%import>
<%args>
AssignmentManager assignmentManager;
int limit = 100;
%args>
-<%java Map rit = assignmentManager
+<%java SortedSet rit = assignmentManager
.getRegionStates().getRegionsInTransitionOrderedByTimestamp(); %>
<%if !rit.isEmpty() %>
<%java>
-List ritsOverThreshold = new ArrayList<>();
-List ritsTwiceThreshold = new ArrayList<>();
+HashSet ritsOverThreshold = new HashSet();
+HashSet ritsTwiceThreshold = new HashSet();
// process the map to find region in transition details
Configuration conf = HBaseConfiguration.create();
int ritThreshold = conf.getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000);
int numOfRITOverThreshold = 0;
long currentTime = System.currentTimeMillis();
-for (Map.Entry e : rit.entrySet()) {
- long ritTime = currentTime - e.getValue().getStamp();
+for (RegionState rs : rit) {
+ long ritTime = currentTime - rs.getStamp();
if(ritTime > (ritThreshold * 2)) {
numOfRITOverThreshold++;
- ritsTwiceThreshold.add(e.getKey());
+ ritsTwiceThreshold.add(rs.getRegion().getEncodedName());
} else if (ritTime > ritThreshold) {
numOfRITOverThreshold++;
- ritsOverThreshold.add(e.getKey());
+ ritsOverThreshold.add(rs.getRegion().getEncodedName());
}
}
@@ -64,7 +60,7 @@ int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
%java>
Regions in Transition
- <% numOfRITs %> region(s) in transition.
+
<% numOfRITs %> region(s) in transition.
<%if !ritsTwiceThreshold.isEmpty() %>
<%elseif !ritsOverThreshold.isEmpty() %>
@@ -72,14 +68,14 @@ int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
<%else>
%if>
- <% numOfRITOverThreshold %> region(s) in transition for
+ <% numOfRITOverThreshold %> region(s) in transition for
more than <% ritThreshold %> milliseconds.
<%java int recordItr = 0; %>
- <%for Map.Entry
entry : rit.entrySet() %>
+ <%for RegionState rs : rit %>
<%if (recordItr % ritsPerPage) == 0 %>
<%if recordItr == 0 %>
@@ -89,18 +85,17 @@ int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
| Region |
State | RIT time (ms) |
%if>
-
- <%if ritsOverThreshold.contains(entry.getKey()) %>
+
+ <%if ritsOverThreshold.contains(rs.getRegion().getEncodedName()) %>
- <%elseif ritsTwiceThreshold.contains(entry.getKey()) %>
+ <%elseif ritsTwiceThreshold.contains(rs.getRegion().getEncodedName()) %>
<%else>
%if>
- | <% entry.getKey() %> |
- <% HRegionInfo.getDescriptiveNameFromRegionStateForDisplay(
- entry.getValue(), conf) %> |
- <% (currentTime - entry.getValue().getStamp()) %> |
+ <% rs.getRegion().getEncodedName() %> |
+ <% HRegionInfo.getDescriptiveNameFromRegionStateForDisplay(rs, conf) %> |
+ <% (currentTime - rs.getStamp()) %> |
<%java recordItr++; %>
<%if (recordItr % ritsPerPage) == 0 %>
@@ -108,7 +103,7 @@ int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
%if>
%for>
-
+
<%if (recordItr % ritsPerPage) != 0 %>
<%for ; (recordItr % ritsPerPage) != 0 ; recordItr++ %>
|
diff --git hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
index f5aa478..82cb4e7 100644
--- hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
+++ hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
@@ -193,14 +193,20 @@ MetricsHBaseServerWrapper mServerWrap;
%args>
- | Compaction Queue Size |
- Flush Queue Size |
- Call Queue Size (bytes) |
+ Compaction Queue Length |
+ Flush Queue Length |
+ Priority Call Queue Length |
+ General Call Queue Length |
+ Replication Call Queue Length |
+ Total Call Queue Size (bytes) |
| <% mWrap.getCompactionQueueSize() %> |
<% mWrap.getFlushQueueSize() %> |
+ <% mServerWrap.getPriorityQueueLength() %> |
+ <% mServerWrap.getGeneralQueueLength() %> |
+ <% mServerWrap.getReplicationQueueLength() %> |
<% TraditionalBinaryPrefix.long2String(mServerWrap.getTotalQueueSize(), "B", 1) %> |
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
index 8b16a5b..33fff97 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
@@ -144,9 +144,6 @@ public class LocalHBaseCluster {
// Always have masters and regionservers come up on port '0' so we don't
// clash over default ports.
conf.set(HConstants.MASTER_PORT, "0");
- if (conf.getInt(HConstants.MASTER_INFO_PORT, 0) != -1) {
- conf.set(HConstants.MASTER_INFO_PORT, "0");
- }
conf.set(HConstants.REGIONSERVER_PORT, "0");
if (conf.getInt(HConstants.REGIONSERVER_INFO_PORT, 0) != -1) {
conf.set(HConstants.REGIONSERVER_INFO_PORT, "0");
@@ -178,7 +175,7 @@ public class LocalHBaseCluster {
Configuration config, final int index)
throws IOException {
// Create each regionserver with its own Configuration instance so each has
- // its HConnection instance rather than share (see HBASE_INSTANCES down in
+ // its Connection instance rather than share (see HBASE_INSTANCES down in
// the guts of ConnectionManager).
// Also, create separate CoordinatedStateManager instance per Server.
@@ -213,7 +210,7 @@ public class LocalHBaseCluster {
public JVMClusterUtil.MasterThread addMaster(Configuration c, final int index)
throws IOException {
// Create each master with its own Configuration instance so each has
- // its HConnection instance rather than share (see HBASE_INSTANCES down in
+ // its Connection instance rather than share (see HBASE_INSTANCES down in
// the guts of ConnectionManager.
// Also, create separate CoordinatedStateManager instance per Server.
@@ -450,4 +447,4 @@ public class LocalHBaseCluster {
connection.close();
cluster.shutdown();
}
-}
\ No newline at end of file
+}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
index 85b1135..63d88ef 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java
@@ -21,10 +21,10 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -44,7 +44,7 @@ class HFileArchiveManager {
private final ZooKeeperWatcher zooKeeper;
private volatile boolean stopped = false;
- public HFileArchiveManager(HConnection connection, Configuration conf)
+ public HFileArchiveManager(Connection connection, Configuration conf)
throws ZooKeeperConnectionException, IOException {
this.zooKeeper = new ZooKeeperWatcher(conf, "hfileArchiveManager-on-" + connection.toString(),
connection);
@@ -74,7 +74,7 @@ class HFileArchiveManager {
*/
public HFileArchiveManager disableHFileBackup(byte[] table) throws KeeperException {
disable(this.zooKeeper, table);
- return this;
+ return this;
}
/**
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java
index 285737d..44e0597 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java
@@ -43,11 +43,11 @@ public class CoprocessorHConnection extends ConnectionImplementation {
private static final NonceGenerator NO_NONCE_GEN = new NoNonceGenerator();
/**
- * Create an {@link HConnection} based on the environment in which we are running the
- * coprocessor. The {@link HConnection} must be externally cleaned up (we bypass the usual HTable
- * cleanup mechanisms since we own everything).
- * @param env environment hosting the {@link HConnection}
- * @return instance of {@link HConnection}.
+ * Create a {@link ClusterConnection} based on the environment in which we are running the
+ * coprocessor. The {@link ClusterConnection} must be externally cleaned up
+ * (we bypass the usual HTable cleanup mechanisms since we own everything).
+ * @param env environment hosting the {@link ClusterConnection}
+ * @return instance of {@link ClusterConnection}.
* @throws IOException if we cannot create the connection
*/
public static ClusterConnection getConnectionForEnvironment(CoprocessorEnvironment env)
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
index 292a935..5da0df7 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
@@ -18,6 +18,11 @@
*/
package org.apache.hadoop.hbase.client;
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -37,11 +42,6 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.io.MultipleIOException;
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
/**
* A wrapper for HTable. Can be used to restrict privilege.
*
@@ -61,7 +61,7 @@ import com.google.protobuf.ServiceException;
public final class HTableWrapper implements Table {
private final Table table;
- private ClusterConnection connection;
+ private final ClusterConnection connection;
private final List openTables;
/**
@@ -134,7 +134,9 @@ public final class HTableWrapper implements Table {
public Boolean[] exists(List gets) throws IOException {
// Do convertion.
boolean [] exists = table.existsAll(gets);
- if (exists == null) return null;
+ if (exists == null) {
+ return null;
+ }
Boolean [] results = new Boolean [exists.length];
for (int i = 0; i < exists.length; i++) {
results[i] = exists[i]? Boolean.TRUE: Boolean.FALSE;
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
index eaf9f43..bff727a 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
@@ -452,6 +452,16 @@ public class BaseRegionObserver implements RegionObserver {
final InternalScanner s) throws IOException {
}
+ @Override
+ public void preReplayWALs(ObserverContext extends RegionCoprocessorEnvironment> env,
+ HRegionInfo info, Path edits) throws IOException {
+ }
+
+ @Override
+ public void postReplayWALs(ObserverContext extends RegionCoprocessorEnvironment> env,
+ HRegionInfo info, Path edits) throws IOException {
+ }
+
/**
* Implementers should override this version of the method and leave the deprecated one as-is.
*/
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
index da0e8b1..e937569 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
@@ -158,9 +158,10 @@ public abstract class CoprocessorHost {
implClass = cl.loadClass(className);
// Add coprocessors as we go to guard against case where a coprocessor is specified twice
// in the configuration
- this.coprocessors.add(loadInstance(implClass, Coprocessor.PRIORITY_SYSTEM, conf));
+ this.coprocessors.add(loadInstance(implClass, priority, conf));
LOG.info("System coprocessor " + className + " was loaded " +
- "successfully with priority (" + priority++ + ").");
+ "successfully with priority (" + priority + ").");
+ ++priority;
} catch (Throwable t) {
// We always abort if system coprocessors cannot be loaded
abortServer(className, t);
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
index 1d985df..4729954 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
@@ -1219,6 +1219,28 @@ public interface RegionObserver extends Coprocessor {
throws IOException;
/**
+ * Called before replaying WALs for this region.
+ * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
+ * effect in this hook.
+ * @param ctx the environment provided by the region server
+ * @param info the RegionInfo for this region
+ * @param edits the file of recovered edits
+ * @throws IOException if an error occurred on the coprocessor
+ */
+ void preReplayWALs(final ObserverContext extends RegionCoprocessorEnvironment> ctx,
+ HRegionInfo info, Path edits) throws IOException;
+
+ /**
+ * Called after replaying WALs for this region.
+ * @param ctx the environment provided by the region server
+ * @param info the RegionInfo for this region
+ * @param edits the file of recovered edits
+ * @throws IOException if an error occurred on the coprocessor
+ */
+ void postReplayWALs(final ObserverContext extends RegionCoprocessorEnvironment> ctx,
+ HRegionInfo info, Path edits) throws IOException;
+
+ /**
* Called before a {@link org.apache.hadoop.hbase.regionserver.wal.WALEdit}
* replayed for this region.
*/
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java hbase-server/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
index 7701a25..1a95c70 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
@@ -72,14 +72,14 @@ public class LogLevel {
System.out.println("Connecting to " + url);
URLConnection connection = url.openConnection();
connection.connect();
-
- BufferedReader in = new BufferedReader(new InputStreamReader(
- connection.getInputStream()));
- for(String line; (line = in.readLine()) != null; )
- if (line.startsWith(MARKER)) {
- System.out.println(TAG.matcher(line).replaceAll(""));
+ try (BufferedReader in = new BufferedReader(new InputStreamReader(
+ connection.getInputStream()))) {
+ for(String line; (line = in.readLine()) != null; ) {
+ if (line.startsWith(MARKER)) {
+ System.out.println(TAG.matcher(line).replaceAll(""));
+ }
}
- in.close();
+ }
} catch (IOException ioe) {
System.err.println("" + ioe);
}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index efc9a30..14a5cd1 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -1712,7 +1712,7 @@ public class HFileBlock implements Cacheable {
ByteBuffer onDiskBlockByteBuffer = ByteBuffer.wrap(onDiskBlock, 0, onDiskSizeWithHeader);
// Verify checksum of the data before using it for building HFileBlock.
if (verifyChecksum &&
- !validateChecksum(offset, onDiskBlockByteBuffer.asReadOnlyBuffer(), hdrSize)) {
+ !validateChecksum(offset, onDiskBlockByteBuffer, hdrSize)) {
return null;
}
// The onDiskBlock will become the headerAndDataBuffer for this block.
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
index e4205eb..3505221 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
@@ -22,6 +22,8 @@ import java.util.List;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
+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.HBaseInterfaceAudience;
@@ -36,6 +38,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
@InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX })
@InterfaceStability.Evolving
public class BalancedQueueRpcExecutor extends RpcExecutor {
+ private static final Log LOG = LogFactory.getLog(BalancedQueueRpcExecutor.class);
protected final List> queues;
private final QueueBalancer balancer;
@@ -62,6 +65,7 @@ public class BalancedQueueRpcExecutor extends RpcExecutor {
queues = new ArrayList>(numQueues);
this.balancer = getBalancer(numQueues);
initializeQueues(numQueues, queueClass, initargs);
+ LOG.debug(name + " queues=" + numQueues + " handlerCount=" + handlerCount);
}
protected void initializeQueues(final int numQueues,
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
index ee36f3f..70d903a 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
@@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hbase.ipc;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.DaemonThreadFactory;
@@ -32,6 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger;
* This can be used for HMaster, where no prioritization is needed.
*/
public class FifoRpcScheduler extends RpcScheduler {
+ private static final Log LOG = LogFactory.getLog(FifoRpcScheduler.class);
private final int handlerCount;
private final int maxQueueLength;
private final AtomicInteger queueSize = new AtomicInteger(0);
@@ -41,6 +44,8 @@ public class FifoRpcScheduler extends RpcScheduler {
this.handlerCount = handlerCount;
this.maxQueueLength = conf.getInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH,
handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
+ LOG.info("Using " + this.getClass().getSimpleName() + " as user call queue; handlerCount=" +
+ handlerCount + "; maxQueueLength=" + maxQueueLength);
}
@Override
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
index 9979c75..4f53709 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
@@ -36,7 +36,7 @@ public class MetricsHBaseServerWrapperImpl implements MetricsHBaseServerWrapper
if (!isServerStarted()) {
return 0;
}
- return server.callQueueSize.get();
+ return server.callQueueSizeInBytes.get();
}
@Override
@@ -65,10 +65,10 @@ public class MetricsHBaseServerWrapperImpl implements MetricsHBaseServerWrapper
@Override
public int getNumOpenConnections() {
- if (!isServerStarted() || this.server.connectionList == null) {
+ if (!isServerStarted()) {
return 0;
}
- return server.connectionList.size();
+ return server.getNumOpenConnections();
}
@Override
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
index 40c11aa..880df36 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.ipc;
import java.util.ArrayList;
import java.util.List;
+import java.util.Locale;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicInteger;
@@ -221,7 +222,7 @@ public abstract class RpcExecutor {
*/
public void resizeQueues(Configuration conf) {
String configKey = RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH;
- if (name != null && name.toLowerCase().contains("priority")) {
+ if (name != null && name.toLowerCase(Locale.ROOT).contains("priority")) {
configKey = RpcScheduler.IPC_SERVER_PRIORITY_MAX_CALLQUEUE_LENGTH;
}
currentQueueLimit = conf.getInt(configKey, currentQueueLimit);
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 483ce86..1087c42 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -48,15 +48,17 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
-import java.util.LinkedList;
import java.util.List;
import java.util.Map;
-import java.util.Random;
import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@@ -113,6 +115,7 @@ import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
@@ -183,11 +186,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
*/
static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10;
- /**
- * The maximum size that we can hold in the RPC queue
- */
- private static final int DEFAULT_MAX_CALLQUEUE_SIZE = 1024 * 1024 * 1024;
-
private final IPCUtil ipcUtil;
private static final String AUTH_FAILED_FOR = "Auth failed for ";
@@ -210,22 +208,30 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
protected int port; // port we listen on
protected InetSocketAddress address; // inet address we listen on
private int readThreads; // number of read threads
- protected int maxIdleTime; // the maximum idle time after
- // which a client may be
- // disconnected
- protected int thresholdIdleConnections; // the number of idle
- // connections after which we
- // will start cleaning up idle
- // connections
- int maxConnectionsToNuke; // the max number of
- // connections to nuke
- // during a cleanup
-
protected MetricsHBaseServer metrics;
protected final Configuration conf;
- private int maxQueueSize;
+ /**
+ * Maximum size in bytes of the currently queued and running Calls. If a new Call puts us over
+ * this size, then we will reject the call (after parsing it though). It will go back to the
+ * client and client will retry. Set this size with "hbase.ipc.server.max.callqueue.size". The
+ * call queue size gets incremented after we parse a call and before we add it to the queue of
+ * calls for the scheduler to use. It get decremented after we have 'run' the Call. The current
+ * size is kept in {@link #callQueueSizeInBytes}.
+ * @see {@link #callQueueSizeInBytes}
+ * @see {@link #DEFAULT_MAX_CALLQUEUE_SIZE}
+ * @see {@link #callQueueSizeInBytes}
+ */
+ private final long maxQueueSizeInBytes;
+ private static final int DEFAULT_MAX_CALLQUEUE_SIZE = 1024 * 1024 * 1024;
+
+ /**
+ * This is a running count of the size in bytes of all outstanding calls whether currently
+ * executing or queued waiting to be run.
+ */
+ protected final Counter callQueueSizeInBytes = new Counter();
+
protected int socketSendBufferSize;
protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
protected final boolean tcpKeepAlive; // if T then use keepalives
@@ -244,19 +250,11 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
*/
volatile boolean started = false;
- /**
- * This is a running count of the size of all outstanding calls by size.
- */
- protected final Counter callQueueSize = new Counter();
-
- protected final List connectionList =
- Collections.synchronizedList(new LinkedList());
- //maintain a list
- //of client connections
+ // maintains the set of client connections and handles idle timeouts
+ private ConnectionManager connectionManager;
private Listener listener = null;
protected Responder responder = null;
protected AuthenticationTokenSecretManager authTokenSecretMgr = null;
- protected int numConnections = 0;
protected HBaseRPCErrorHandler errorHandler = null;
@@ -623,18 +621,16 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
private Selector selector = null; //the selector that we use for the server
private Reader[] readers = null;
private int currentReader = 0;
- private Random rand = new Random();
- private long lastCleanupRunTime = 0; //the last time when a cleanup connec-
- //-tion (for idle connections) ran
- private long cleanupInterval = 10000; //the minimum interval between
- //two cleanup runs
- private int backlogLength;
+ private final int readerPendingConnectionQueueLength;
private ExecutorService readPool;
public Listener(final String name) throws IOException {
super(name);
- backlogLength = conf.getInt("hbase.ipc.server.listen.queue.size", 128);
+ // The backlog of requests that we will have the serversocket carry.
+ int backlogLength = conf.getInt("hbase.ipc.server.listen.queue.size", 128);
+ readerPendingConnectionQueueLength =
+ conf.getInt("hbase.ipc.server.read.connection-queue.size", 100);
// Create a new server socket and set to non blocking mode
acceptChannel = ServerSocketChannel.open();
acceptChannel.configureBlocking(false);
@@ -644,9 +640,11 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
address = (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
// create a selector;
- selector= Selector.open();
+ selector = Selector.open();
readers = new Reader[readThreads];
+ // Why this executor thing? Why not like hadoop just start up all the threads? I suppose it
+ // has an advantage in that it is easy to shutdown the pool.
readPool = Executors.newFixedThreadPool(readThreads,
new ThreadFactoryBuilder().setNameFormat(
"RpcServer.reader=%d,bindAddress=" + bindAddress.getHostName() +
@@ -667,12 +665,15 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
private class Reader implements Runnable {
- private volatile boolean adding = false;
+ final private LinkedBlockingQueue pendingConnections;
private final Selector readSelector;
Reader() throws IOException {
+ this.pendingConnections =
+ new LinkedBlockingQueue(readerPendingConnectionQueueLength);
this.readSelector = Selector.open();
}
+
@Override
public void run() {
try {
@@ -689,11 +690,14 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
private synchronized void doRunLoop() {
while (running) {
try {
- readSelector.select();
- while (adding) {
- this.wait(1000);
+ // Consume as many connections as currently queued to avoid
+ // unbridled acceptance of connections that starves the select
+ int size = pendingConnections.size();
+ for (int i=size; i>0; i--) {
+ Connection conn = pendingConnections.take();
+ conn.channel.register(readSelector, SelectionKey.OP_READ, conn);
}
-
+ readSelector.select();
Iterator iter = readSelector.selectedKeys().iterator();
while (iter.hasNext()) {
SelectionKey key = iter.next();
@@ -703,9 +707,12 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
doRead(key);
}
}
+ key = null;
}
} catch (InterruptedException e) {
- LOG.debug("Interrupted while sleeping");
+ if (running) { // unexpected -- log it
+ LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
+ }
return;
} catch (IOException ex) {
LOG.info(getName() + ": IOException in Reader", ex);
@@ -714,76 +721,14 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
}
/**
- * This gets reader into the state that waits for the new channel
- * to be registered with readSelector. If it was waiting in select()
- * the thread will be woken up, otherwise whenever select() is called
- * it will return even if there is nothing to read and wait
- * in while(adding) for finishAdd call
+ * Updating the readSelector while it's being used is not thread-safe,
+ * so the connection must be queued. The reader will drain the queue
+ * and update its readSelector before performing the next select
*/
- public void startAdd() {
- adding = true;
+ public void addConnection(Connection conn) throws IOException {
+ pendingConnections.add(conn);
readSelector.wakeup();
}
-
- public synchronized SelectionKey registerChannel(SocketChannel channel)
- throws IOException {
- return channel.register(readSelector, SelectionKey.OP_READ);
- }
-
- public synchronized void finishAdd() {
- adding = false;
- this.notify();
- }
- }
-
- /** cleanup connections from connectionList. Choose a random range
- * to scan and also have a limit on the number of the connections
- * that will be cleanedup per run. The criteria for cleanup is the time
- * for which the connection was idle. If 'force' is true then all
- * connections will be looked at for the cleanup.
- * @param force all connections will be looked at for cleanup
- */
- private void cleanupConnections(boolean force) {
- if (force || numConnections > thresholdIdleConnections) {
- long currentTime = System.currentTimeMillis();
- if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
- return;
- }
- int start = 0;
- int end = numConnections - 1;
- if (!force) {
- start = rand.nextInt() % numConnections;
- end = rand.nextInt() % numConnections;
- int temp;
- if (end < start) {
- temp = start;
- start = end;
- end = temp;
- }
- }
- int i = start;
- int numNuked = 0;
- while (i <= end) {
- Connection c;
- synchronized (connectionList) {
- try {
- c = connectionList.get(i);
- } catch (Exception e) {return;}
- }
- if (c.timedOut(currentTime)) {
- if (LOG.isDebugEnabled())
- LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
- closeConnection(c);
- numNuked++;
- end--;
- //noinspection UnusedAssignment
- c = null;
- if (!force && numNuked == maxConnectionsToNuke) break;
- }
- else i++;
- }
- lastCleanupRunTime = System.currentTimeMillis();
- }
}
@Override
@@ -792,6 +737,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
"it will have per impact")
public void run() {
LOG.info(getName() + ": starting");
+ connectionManager.startIdleScan();
while (running) {
SelectionKey key = null;
try {
@@ -815,7 +761,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
if (errorHandler.checkOOME(e)) {
LOG.info(getName() + ": exiting on OutOfMemoryError");
closeCurrentConnection(key, e);
- cleanupConnections(true);
+ connectionManager.closeIdle(true);
return;
}
} else {
@@ -824,22 +770,18 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
// some thread(s) a chance to finish
LOG.warn(getName() + ": OutOfMemoryError in server select", e);
closeCurrentConnection(key, e);
- cleanupConnections(true);
+ connectionManager.closeIdle(true);
try {
Thread.sleep(60000);
} catch (InterruptedException ex) {
LOG.debug("Interrupted while sleeping");
- return;
}
}
} catch (Exception e) {
closeCurrentConnection(key, e);
}
- cleanupConnections(false);
}
-
LOG.info(getName() + ": stopping");
-
synchronized (this) {
try {
acceptChannel.close();
@@ -851,10 +793,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
selector= null;
acceptChannel= null;
- // clean up all connections
- while (!connectionList.isEmpty()) {
- closeConnection(connectionList.remove(0));
- }
+ // close all connections
+ connectionManager.stopIdleScan();
+ connectionManager.closeAll();
}
}
@@ -862,10 +803,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
if (key != null) {
Connection c = (Connection)key.attachment();
if (c != null) {
- if (LOG.isDebugEnabled()) {
- LOG.debug(getName() + ": disconnecting client " + c.getHostAddress() +
- (e != null ? " on error " + e.getMessage() : ""));
- }
closeConnection(c);
key.attach(null);
}
@@ -876,37 +813,24 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
return address;
}
- void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
- Connection c;
+ void doAccept(SelectionKey key) throws InterruptedException, IOException, OutOfMemoryError {
ServerSocketChannel server = (ServerSocketChannel) key.channel();
-
SocketChannel channel;
while ((channel = server.accept()) != null) {
- try {
- channel.configureBlocking(false);
- channel.socket().setTcpNoDelay(tcpNoDelay);
- channel.socket().setKeepAlive(tcpKeepAlive);
- } catch (IOException ioe) {
- channel.close();
- throw ioe;
- }
-
+ channel.configureBlocking(false);
+ channel.socket().setTcpNoDelay(tcpNoDelay);
+ channel.socket().setKeepAlive(tcpKeepAlive);
Reader reader = getReader();
- try {
- reader.startAdd();
- SelectionKey readKey = reader.registerChannel(channel);
- c = getConnection(channel, System.currentTimeMillis());
- readKey.attach(c);
- synchronized (connectionList) {
- connectionList.add(numConnections, c);
- numConnections++;
+ Connection c = connectionManager.register(channel);
+ // If the connectionManager can't take it, close the connection.
+ if (c == null) {
+ if (channel.isOpen()) {
+ IOUtils.cleanup(null, channel);
}
- if (LOG.isDebugEnabled())
- LOG.debug(getName() + ": connection from " + c.toString() +
- "; # active connections: " + numConnections);
- } finally {
- reader.finishAdd();
+ continue;
}
+ key.attach(c); // so closeCurrentConnection can get the object
+ reader.addConnection(c);
}
}
@@ -919,12 +843,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
c.setLastContact(System.currentTimeMillis());
try {
count = c.readAndProcess();
-
- if (count > 0) {
- c.setLastContact(System.currentTimeMillis());
- }
-
} catch (InterruptedException ieo) {
+ LOG.info(Thread.currentThread().getName() + ": readAndProcess caught InterruptedException", ieo);
throw ieo;
} catch (Exception e) {
if (LOG.isDebugEnabled()) {
@@ -933,12 +853,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
count = -1; //so that the (count < 0) block is executed
}
if (count < 0) {
- if (LOG.isDebugEnabled()) {
- LOG.debug(getName() + ": DISCONNECTING client " + c.toString() +
- " because read count=" + count +
- ". Number of active connections: " + numConnections);
- }
closeConnection(c);
+ c = null;
+ } else {
+ c.setLastContact(System.currentTimeMillis());
}
}
@@ -957,6 +875,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
readPool.shutdownNow();
}
+ synchronized Selector getSelector() { return selector; }
+
// The method that will return the next reader to work with
// Simplistic implementation of round robin for now
Reader getReader() {
@@ -1355,6 +1275,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
return null;
}
+ public long getLastContact() {
+ return lastContact;
+ }
+
/* Return true if the connection has no outstanding rpc */
private boolean isIdle() {
return rpcCount.get() == 0;
@@ -1370,10 +1294,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
rpcCount.increment();
}
- protected boolean timedOut(long currentTime) {
- return isIdle() && currentTime - lastContact > maxIdleTime;
- }
-
private UserGroupInformation getAuthorizedUgi(String authorizedId)
throws IOException {
UserGroupInformation authorizedUgi;
@@ -1883,7 +1803,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
}
// Enforcing the call queue size, this triggers a retry in the client
// This is a bit late to be doing this check - we have already read in the total request.
- if ((totalRequestSize + callQueueSize.get()) > maxQueueSize) {
+ if ((totalRequestSize + callQueueSizeInBytes.get()) > maxQueueSizeInBytes) {
final Call callTooBig =
new Call(id, this.service, null, null, null, null, this,
responder, totalRequestSize, null, null, 0);
@@ -1954,7 +1874,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
totalRequestSize, traceInfo, this.addr, timeout);
if (!scheduler.dispatch(new CallRunner(RpcServer.this, call))) {
- callQueueSize.add(-1 * call.getSize());
+ callQueueSizeInBytes.add(-1 * call.getSize());
ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
metrics.exception(CALL_QUEUE_TOO_BIG_EXCEPTION);
@@ -2093,12 +2013,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
this.bindAddress = bindAddress;
this.conf = conf;
this.socketSendBufferSize = 0;
- this.maxQueueSize =
- this.conf.getInt("hbase.ipc.server.max.callqueue.size", DEFAULT_MAX_CALLQUEUE_SIZE);
+ // See declaration above for documentation on what this size is.
+ this.maxQueueSizeInBytes =
+ this.conf.getLong("hbase.ipc.server.max.callqueue.size", DEFAULT_MAX_CALLQUEUE_SIZE);
this.readThreads = conf.getInt("hbase.ipc.server.read.threadpool.size", 10);
- this.maxIdleTime = 2 * conf.getInt("hbase.ipc.client.connection.maxidletime", 1000);
- this.maxConnectionsToNuke = conf.getInt("hbase.ipc.client.kill.max", 10);
- this.thresholdIdleConnections = conf.getInt("hbase.ipc.client.idlethreshold", 4000);
this.purgeTimeout = conf.getLong("hbase.ipc.client.call.purge.timeout",
2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
this.warnResponseTime = conf.getInt(WARN_RESPONSE_TIME, DEFAULT_WARN_RESPONSE_TIME);
@@ -2120,6 +2038,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
// Create the responder here
responder = new Responder();
+ connectionManager = new ConnectionManager();
this.authorize = conf.getBoolean(HADOOP_SECURITY_AUTHORIZATION, false);
this.userProvider = UserProvider.instantiate(conf);
this.isSecurityEnabled = userProvider.isHBaseSecurityEnabled();
@@ -2177,12 +2096,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
}
protected void closeConnection(Connection connection) {
- synchronized (connectionList) {
- if (connectionList.remove(connection)) {
- numConnections--;
- }
- }
- connection.close();
+ connectionManager.close(connection);
}
Configuration getConf() {
@@ -2440,7 +2354,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
@Override
public void addCallSize(final long diff) {
- this.callQueueSize.add(diff);
+ this.callQueueSizeInBytes.add(diff);
}
/**
@@ -2578,6 +2492,14 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
}
/**
+ * The number of open RPC conections
+ * @return the number of open rpc connections
+ */
+ public int getNumOpenConnections() {
+ return connectionManager.size();
+ }
+
+ /**
* Returns the username for any user associated with the current RPC
* request or null if no user is set.
*/
@@ -2695,4 +2617,149 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
public RpcScheduler getScheduler() {
return scheduler;
}
+
+ private class ConnectionManager {
+ final private AtomicInteger count = new AtomicInteger();
+ final private Set connections;
+
+ final private Timer idleScanTimer;
+ final private int idleScanThreshold;
+ final private int idleScanInterval;
+ final private int maxIdleTime;
+ final private int maxIdleToClose;
+
+ ConnectionManager() {
+ this.idleScanTimer = new Timer("RpcServer idle connection scanner for port " + port, true);
+ this.idleScanThreshold = conf.getInt("hbase.ipc.client.idlethreshold", 4000);
+ this.idleScanInterval =
+ conf.getInt("hbase.ipc.client.connection.idle-scan-interval.ms", 10000);
+ this.maxIdleTime = 2 * conf.getInt("hbase.ipc.client.connection.maxidletime", 10000);
+ this.maxIdleToClose = conf.getInt("hbase.ipc.client.kill.max", 10);
+ int handlerCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,
+ HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT);
+ int maxConnectionQueueSize =
+ handlerCount * conf.getInt("hbase.ipc.server.handler.queue.size", 100);
+ // create a set with concurrency -and- a thread-safe iterator, add 2
+ // for listener and idle closer threads
+ this.connections = Collections.newSetFromMap(
+ new ConcurrentHashMap(
+ maxConnectionQueueSize, 0.75f, readThreads+2));
+ }
+
+ private boolean add(Connection connection) {
+ boolean added = connections.add(connection);
+ if (added) {
+ count.getAndIncrement();
+ }
+ return added;
+ }
+
+ private boolean remove(Connection connection) {
+ boolean removed = connections.remove(connection);
+ if (removed) {
+ count.getAndDecrement();
+ }
+ return removed;
+ }
+
+ int size() {
+ return count.get();
+ }
+
+ Connection[] toArray() {
+ return connections.toArray(new Connection[0]);
+ }
+
+ Connection register(SocketChannel channel) {
+ Connection connection = new Connection(channel, System.currentTimeMillis());
+ add(connection);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Server connection from " + connection +
+ "; connections=" + size() +
+ ", queued calls size (bytes)=" + callQueueSizeInBytes.get() +
+ ", general queued calls=" + scheduler.getGeneralQueueLength() +
+ ", priority queued calls=" + scheduler.getPriorityQueueLength());
+ }
+ return connection;
+ }
+
+ boolean close(Connection connection) {
+ boolean exists = remove(connection);
+ if (exists) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(Thread.currentThread().getName() +
+ ": disconnecting client " + connection +
+ ". Number of active connections: "+ size());
+ }
+ // only close if actually removed to avoid double-closing due
+ // to possible races
+ connection.close();
+ }
+ return exists;
+ }
+
+ // synch'ed to avoid explicit invocation upon OOM from colliding with
+ // timer task firing
+ synchronized void closeIdle(boolean scanAll) {
+ long minLastContact = System.currentTimeMillis() - maxIdleTime;
+ // concurrent iterator might miss new connections added
+ // during the iteration, but that's ok because they won't
+ // be idle yet anyway and will be caught on next scan
+ int closed = 0;
+ for (Connection connection : connections) {
+ // stop if connections dropped below threshold unless scanning all
+ if (!scanAll && size() < idleScanThreshold) {
+ break;
+ }
+ // stop if not scanning all and max connections are closed
+ if (connection.isIdle() &&
+ connection.getLastContact() < minLastContact &&
+ close(connection) &&
+ !scanAll && (++closed == maxIdleToClose)) {
+ break;
+ }
+ }
+ }
+
+ void closeAll() {
+ // use a copy of the connections to be absolutely sure the concurrent
+ // iterator doesn't miss a connection
+ for (Connection connection : toArray()) {
+ close(connection);
+ }
+ }
+
+ void startIdleScan() {
+ scheduleIdleScanTask();
+ }
+
+ void stopIdleScan() {
+ idleScanTimer.cancel();
+ }
+
+ private void scheduleIdleScanTask() {
+ if (!running) {
+ return;
+ }
+ TimerTask idleScanTask = new TimerTask(){
+ @Override
+ public void run() {
+ if (!running) {
+ return;
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(Thread.currentThread().getName()+": task running");
+ }
+ try {
+ closeIdle(false);
+ } finally {
+ // explicitly reschedule so next execution occurs relative
+ // to the end of this scan, not the beginning
+ scheduleIdleScanTask();
+ }
+ }
+ };
+ idleScanTimer.schedule(idleScanTask, idleScanInterval);
+ }
+ }
}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
index 431aeeb..d9d61c1 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
@@ -34,8 +34,11 @@ import org.apache.hadoop.hbase.conf.ConfigurationObserver;
import org.apache.hadoop.hbase.util.BoundedPriorityBlockingQueue;
/**
- * A scheduler that maintains isolated handler pools for general,
- * high-priority, and replication requests.
+ * The default scheduler. Configurable. Maintains isolated handler pools for general ('default'),
+ * high-priority ('priority'), and replication ('replication') requests. Default behavior is to
+ * balance the requests across handlers. Add configs to enable balancing by read vs writes, etc.
+ * See below article for explanation of options.
+ * @see Overview on Request Queuing
*/
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
@InterfaceStability.Evolving
@@ -49,7 +52,8 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
public static final String CALL_QUEUE_HANDLER_FACTOR_CONF_KEY =
"hbase.ipc.server.callqueue.handler.factor";
- /** If set to 'deadline', uses a priority queue and deprioritize long-running scans */
+ /** If set to 'deadline', the default, uses a priority queue and deprioritizes long-running scans
+ */
public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type";
public static final String CALL_QUEUE_TYPE_CODEL_CONF_VALUE = "codel";
public static final String CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE = "deadline";
@@ -190,54 +194,58 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
float callQueuesHandlersFactor = conf.getFloat(CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 0);
int numCallQueues = Math.max(1, (int)Math.round(handlerCount * callQueuesHandlersFactor));
-
- LOG.info("Using " + callQueueType + " as user call queue, count=" + numCallQueues);
-
+ LOG.info("Using " + callQueueType + " as user call queue; numCallQueues=" + numCallQueues +
+ "; callQReadShare=" + callqReadShare + ", callQScanShare=" + callqScanShare);
if (numCallQueues > 1 && callqReadShare > 0) {
// multiple read/write queues
- if (callQueueType.equals(CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE)) {
+ if (isDeadlineQueueType(callQueueType)) {
CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
- callExecutor = new RWQueueRpcExecutor("RW.default", handlerCount, numCallQueues,
+ callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount, numCallQueues,
callqReadShare, callqScanShare, maxQueueLength, conf, abortable,
BoundedPriorityBlockingQueue.class, callPriority);
} else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
Object[] callQueueInitArgs = {maxQueueLength, codelTargetDelay, codelInterval,
codelLifoThreshold, numGeneralCallsDropped, numLifoModeSwitches};
- callExecutor = new RWQueueRpcExecutor("RW.default", handlerCount,
+ callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount,
numCallQueues, callqReadShare, callqScanShare,
AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs,
AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs);
} else {
- callExecutor = new RWQueueRpcExecutor("RW.default", handlerCount, numCallQueues,
+ callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount, numCallQueues,
callqReadShare, callqScanShare, maxQueueLength, conf, abortable);
}
} else {
// multiple queues
- if (callQueueType.equals(CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE)) {
+ if (isDeadlineQueueType(callQueueType)) {
CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
- callExecutor = new BalancedQueueRpcExecutor("B.default", handlerCount, numCallQueues,
- conf, abortable, BoundedPriorityBlockingQueue.class, maxQueueLength, callPriority);
+ callExecutor =
+ new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount, numCallQueues,
+ conf, abortable, BoundedPriorityBlockingQueue.class, maxQueueLength, callPriority);
} else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
- callExecutor = new BalancedQueueRpcExecutor("B.default", handlerCount, numCallQueues,
- conf, abortable, AdaptiveLifoCoDelCallQueue.class, maxQueueLength,
- codelTargetDelay, codelInterval, codelLifoThreshold,
- numGeneralCallsDropped, numLifoModeSwitches);
+ callExecutor =
+ new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount, numCallQueues,
+ conf, abortable, AdaptiveLifoCoDelCallQueue.class, maxQueueLength,
+ codelTargetDelay, codelInterval, codelLifoThreshold,
+ numGeneralCallsDropped, numLifoModeSwitches);
} else {
- callExecutor = new BalancedQueueRpcExecutor("B.default", handlerCount,
+ callExecutor = new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount,
numCallQueues, maxQueueLength, conf, abortable);
}
}
-
// Create 2 queues to help priorityExecutor be more scalable.
this.priorityExecutor = priorityHandlerCount > 0 ?
- new BalancedQueueRpcExecutor("Priority", priorityHandlerCount, 2, maxPriorityQueueLength) :
- null;
-
+ new BalancedQueueRpcExecutor("BalancedQ.priority", priorityHandlerCount, 2,
+ maxPriorityQueueLength):
+ null;
this.replicationExecutor =
- replicationHandlerCount > 0 ? new BalancedQueueRpcExecutor("Replication",
+ replicationHandlerCount > 0 ? new BalancedQueueRpcExecutor("BalancedQ.replication",
replicationHandlerCount, 1, maxQueueLength, conf, abortable) : null;
}
+ private static boolean isDeadlineQueueType(final String callQueueType) {
+ return callQueueType.equals(CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
+ }
+
public SimpleRpcScheduler(
Configuration conf,
int handlerCount,
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index b7f67e6..7668ac9 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -28,6 +28,7 @@ import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
+import java.util.Locale;
import java.util.Map;
import java.util.TreeMap;
import java.util.UUID;
@@ -401,7 +402,7 @@ public class Import extends Configured implements Tool {
filter = instantiateFilter(conf);
String durabilityStr = conf.get(WAL_DURABILITY);
if(durabilityStr != null){
- durability = Durability.valueOf(durabilityStr.toUpperCase());
+ durability = Durability.valueOf(durabilityStr.toUpperCase(Locale.ROOT));
LOG.info("setting WAL durability to " + durability);
} else {
LOG.info("setting WAL durability to default.");
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/JarFinder.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/JarFinder.java
index dfbe648..cf334db 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/JarFinder.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/JarFinder.java
@@ -124,8 +124,9 @@ public class JarFinder {
jarDir));
}
}
- JarOutputStream zos = new JarOutputStream(new FileOutputStream(jarFile));
- jarDir(dir, "", zos);
+ try (JarOutputStream zos = new JarOutputStream(new FileOutputStream(jarFile))) {
+ jarDir(dir, "", zos);
+ }
}
/**
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 0084878..a23d739 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -25,6 +25,32 @@ import com.google.common.collect.Multimap;
import com.google.common.collect.Multimaps;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
import org.apache.commons.lang.mutable.MutableInt;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -45,7 +71,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.RegionServerCallable;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
@@ -76,32 +101,6 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
/**
* Tool to load the output of HFileOutputFormat into an existing table.
* @see #usage()
@@ -165,7 +164,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
+ "\n");
}
- private static interface BulkHFileVisitor {
+ private interface BulkHFileVisitor {
TFamily bulkFamily(final byte[] familyName)
throws IOException;
void bulkHFile(final TFamily family, final FileStatus hfileStatus)
@@ -308,25 +307,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* pre-existing table. This method is not threadsafe.
*
* @param hfofDir the directory that was provided as the output path
- * of a job using HFileOutputFormat
- * @param table the table to load into
- * @throws TableNotFoundException if table does not yet exist
- */
- @SuppressWarnings("deprecation")
- public void doBulkLoad(Path hfofDir, final HTable table)
- throws TableNotFoundException, IOException {
- try (Admin admin = table.getConnection().getAdmin();
- RegionLocator rl = table.getRegionLocator()) {
- doBulkLoad(hfofDir, admin, table, rl);
- }
- }
-
- /**
- * Perform a bulk load of the given directory into the given
- * pre-existing table. This method is not threadsafe.
- *
- * @param hfofDir the directory that was provided as the output path
- * of a job using HFileOutputFormat
+ * of a job using HFileOutputFormat
* @param table the table to load into
* @throws TableNotFoundException if table does not yet exist
*/
@@ -341,7 +322,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
// LQI queue does not need to be threadsafe -- all operations on this queue
// happen in this thread
- Deque queue = new LinkedList();
+ Deque queue = new LinkedList<>();
try {
/*
* Checking hfile format is a time-consuming operation, we should have an option to skip
@@ -426,8 +407,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
}
if (queue != null && !queue.isEmpty()) {
- throw new RuntimeException("Bulk load aborted with some files not yet loaded."
- + "Please check log for more details.");
+ throw new RuntimeException("Bulk load aborted with some files not yet loaded."
+ + "Please check log for more details.");
}
}
@@ -463,7 +444,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
private void validateFamiliesInHFiles(Table table, Deque queue)
throws IOException {
Collection families = table.getTableDescriptor().getFamilies();
- List familyNames = new ArrayList(families.size());
+ List familyNames = new ArrayList<>(families.size());
for (HColumnDescriptor family : families) {
familyNames.add(family.getNameAsString());
}
@@ -520,7 +501,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
ExecutorService pool, Deque queue,
final Multimap regionGroups) throws IOException {
// atomically bulk load the groups.
- Set>> loadingFutures = new HashSet>>();
+ Set>> loadingFutures = new HashSet<>();
for (Entry> e: regionGroups.asMap().entrySet()){
final byte[] first = e.getKey().array();
final Collection lqis = e.getValue();
@@ -563,9 +544,9 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
private boolean checkHFilesCountPerRegionPerFamily(
final Multimap regionGroups) {
for (Entry> e: regionGroups.asMap().entrySet()) {
+ ? extends Collection> e: regionGroups.asMap().entrySet()) {
final Collection lqis = e.getValue();
- HashMap filesMap = new HashMap();
+ HashMap filesMap = new HashMap<>();
for (LoadQueueItem lqi: lqis) {
MutableInt count = filesMap.get(lqi.family);
if (count == null) {
@@ -597,7 +578,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
final Multimap regionGroups = Multimaps.synchronizedMultimap(rgs);
// drain LQIs and figure out bulk load groups
- Set>> splittingFutures = new HashSet>>();
+ Set>> splittingFutures = new HashSet<>();
while (!queue.isEmpty()) {
final LoadQueueItem item = queue.remove();
@@ -650,7 +631,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
}
LOG.info("HFile at " + hfilePath + " no longer fits inside a single " +
- "region. Splitting...");
+ "region. Splitting...");
String uniqueName = getUniqueName();
HColumnDescriptor familyDesc = table.getTableDescriptor().getFamily(item.family);
@@ -692,7 +673,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* LQI's corresponding to the resultant hfiles.
*
* protected for testing
- * @throws IOException
+ * @throws IOException if an IO failure is encountered
*/
protected List groupOrSplit(Multimap regionGroups,
final LoadQueueItem item, final Table table,
@@ -786,13 +767,13 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* Protected for testing.
*
* @return empty list if success, list of items to retry on recoverable
- * failure
+ * failure
*/
protected List tryAtomicRegionLoad(final Connection conn,
final TableName tableName, final byte[] first, final Collection lqis)
throws IOException {
final List> famPaths =
- new ArrayList>(lqis.size());
+ new ArrayList<>(lqis.size());
for (LoadQueueItem lqi : lqis) {
famPaths.add(Pair.newPair(lqi.family, lqi.hfilePath.toString()));
}
@@ -857,7 +838,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
};
try {
- List toRetry = new ArrayList();
+ List toRetry = new ArrayList<>();
Configuration conf = getConf();
boolean success = RpcRetryingCallerFactory.instantiate(conf,
null). newCaller()
@@ -890,8 +871,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
static void splitStoreFile(
Configuration conf, Path inFile,
HColumnDescriptor familyDesc, byte[] splitKey,
- Path bottomOut, Path topOut) throws IOException
- {
+ Path bottomOut, Path topOut) throws IOException {
// Open reader with no block cache, and not in-memory
Reference topReference = Reference.createTopReference(splitKey);
Reference bottomReference = Reference.createBottomReference(splitKey);
@@ -944,8 +924,12 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
}
}
} finally {
- if (halfWriter != null) halfWriter.close();
- if (halfReader != null) halfReader.close(cacheConf.shouldEvictOnClose());
+ if (halfWriter != null) {
+ halfWriter.close();
+ }
+ if (halfReader != null) {
+ halfReader.close(cacheConf.shouldEvictOnClose());
+ }
}
}
@@ -972,16 +956,20 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* 2) Return the boundary list.
*/
public static byte[][] inferBoundaries(TreeMap bdryMap) {
- ArrayList keysArray = new ArrayList();
+ ArrayList keysArray = new ArrayList<>();
int runningValue = 0;
byte[] currStartKey = null;
boolean firstBoundary = true;
for (Map.Entry item: bdryMap.entrySet()) {
- if (runningValue == 0) currStartKey = item.getKey();
+ if (runningValue == 0) {
+ currStartKey = item.getKey();
+ }
runningValue += item.getValue();
if (runningValue == 0) {
- if (!firstBoundary) keysArray.add(currStartKey);
+ if (!firstBoundary) {
+ keysArray.add(currStartKey);
+ }
firstBoundary = false;
}
}
@@ -1000,7 +988,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
// Add column families
// Build a set of keys
final HTableDescriptor htd = new HTableDescriptor(tableName);
- final TreeMap map = new TreeMap(Bytes.BYTES_COMPARATOR);
+ final TreeMap map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
visitBulkHFiles(fs, hfofDir, new BulkHFileVisitor() {
@Override
public HColumnDescriptor bulkFamily(final byte[] familyName) {
@@ -1073,8 +1061,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
Path hfofDir = new Path(dirPath);
try (Table table = connection.getTable(tableName);
- RegionLocator locator = connection.getRegionLocator(tableName)) {
- doBulkLoad(hfofDir, admin, table, locator);
+ RegionLocator locator = connection.getRegionLocator(tableName)) {
+ doBulkLoad(hfofDir, admin, table, locator);
}
}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java
index be20d90..7ad68ea 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.mapreduce;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
+import java.util.Locale;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -253,7 +254,7 @@ implements Configurable {
@Override
public List getSplits(JobContext context) throws IOException {
List splits = super.getSplits(context);
- if ((conf.get(SHUFFLE_MAPS) != null) && "true".equals(conf.get(SHUFFLE_MAPS).toLowerCase())) {
+ if ((conf.get(SHUFFLE_MAPS) != null) && "true".equals(conf.get(SHUFFLE_MAPS).toLowerCase(Locale.ROOT))) {
Collections.shuffle(splits);
}
return splits;
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index a452036..655c71a 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.mapreduce.replication;
import java.io.IOException;
+import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -35,6 +36,9 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
@@ -77,6 +81,7 @@ public class VerifyReplication extends Configured implements Tool {
static String tableName = null;
static String families = null;
static String peerId = null;
+ static String rowPrefixes = null;
private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
@@ -123,6 +128,8 @@ public class VerifyReplication extends Configured implements Tool {
scan.addFamily(Bytes.toBytes(fam));
}
}
+ String rowPrefixes = conf.get(NAME + ".rowPrefixes", null);
+ setRowPrefixFilter(scan, rowPrefixes);
scan.setTimeRange(startTime, endTime);
int versions = conf.getInt(NAME+".versions", -1);
LOG.info("Setting number of version inside map as: " + versions);
@@ -271,6 +278,9 @@ public class VerifyReplication extends Configured implements Tool {
if (families != null) {
conf.set(NAME+".families", families);
}
+ if (rowPrefixes != null){
+ conf.set(NAME+".rowPrefixes", rowPrefixes);
+ }
Pair peerConfigPair = getPeerQuorumConfig(conf);
ReplicationPeerConfig peerConfig = peerConfigPair.getFirst();
@@ -299,6 +309,9 @@ public class VerifyReplication extends Configured implements Tool {
scan.addFamily(Bytes.toBytes(fam));
}
}
+
+ setRowPrefixFilter(scan, rowPrefixes);
+
TableMapReduceUtil.initTableMapperJob(tableName, scan,
Verifier.class, null, null, job);
@@ -311,11 +324,38 @@ public class VerifyReplication extends Configured implements Tool {
return job;
}
+ private static void setRowPrefixFilter(Scan scan, String rowPrefixes) {
+ if (rowPrefixes != null && !rowPrefixes.isEmpty()) {
+ String[] rowPrefixArray = rowPrefixes.split(",");
+ Arrays.sort(rowPrefixArray);
+ FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
+ for (String prefix : rowPrefixArray) {
+ Filter filter = new PrefixFilter(Bytes.toBytes(prefix));
+ filterList.addFilter(filter);
+ }
+ scan.setFilter(filterList);
+ byte[] startPrefixRow = Bytes.toBytes(rowPrefixArray[0]);
+ byte[] lastPrefixRow = Bytes.toBytes(rowPrefixArray[rowPrefixArray.length -1]);
+ setStartAndStopRows(scan, startPrefixRow, lastPrefixRow);
+ }
+ }
+
+ private static void setStartAndStopRows(Scan scan, byte[] startPrefixRow, byte[] lastPrefixRow) {
+ scan.setStartRow(startPrefixRow);
+ byte[] stopRow = Bytes.add(Bytes.head(lastPrefixRow, lastPrefixRow.length - 1),
+ new byte[]{(byte) (lastPrefixRow[lastPrefixRow.length - 1] + 1)});
+ scan.setStopRow(stopRow);
+ }
+
private static boolean doCommandLine(final String[] args) {
if (args.length < 2) {
printUsage(null);
return false;
}
+ //in case we've been run before, restore all parameters to their initial states
+ //Otherwise, if our previous run included a parameter not in args this time,
+ //we might hold on to the old value.
+ restoreDefaults();
try {
for (int i = 0; i < args.length; i++) {
String cmd = args[i];
@@ -354,6 +394,12 @@ public class VerifyReplication extends Configured implements Tool {
continue;
}
+ final String rowPrefixesKey = "--row-prefixes=";
+ if (cmd.startsWith(rowPrefixesKey)){
+ rowPrefixes = cmd.substring(rowPrefixesKey.length());
+ continue;
+ }
+
if (i == args.length-2) {
peerId = cmd;
}
@@ -370,6 +416,17 @@ public class VerifyReplication extends Configured implements Tool {
return true;
}
+ private static void restoreDefaults() {
+ startTime = 0;
+ endTime = Long.MAX_VALUE;
+ batch = Integer.MAX_VALUE;
+ versions = -1;
+ tableName = null;
+ families = null;
+ peerId = null;
+ rowPrefixes = null;
+ }
+
/*
* @param errorMsg Error message. Can be null.
*/
@@ -378,7 +435,7 @@ public class VerifyReplication extends Configured implements Tool {
System.err.println("ERROR: " + errorMsg);
}
System.err.println("Usage: verifyrep [--starttime=X]" +
- " [--stoptime=Y] [--families=A] ");
+ " [--stoptime=Y] [--families=A] [--row-prefixes=B] ");
System.err.println();
System.err.println("Options:");
System.err.println(" starttime beginning of the time range");
@@ -386,6 +443,7 @@ public class VerifyReplication extends Configured implements Tool {
System.err.println(" endtime end of the time range");
System.err.println(" versions number of cell versions to verify");
System.err.println(" families comma-separated list of families to copy");
+ System.err.println(" row-prefixes comma-separated list of row key prefixes to filter on ");
System.err.println();
System.err.println("Args:");
System.err.println(" peerid Id of the peer used for verification, must match the one given for replication");
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index bcf7b7a..f16463f 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -471,9 +471,9 @@ public class AssignmentManager {
}
if (!failover) {
// If any region except meta is in transition on a live server, it's a failover.
- Map regionsInTransition = regionStates.getRegionsInTransition();
+ Set regionsInTransition = regionStates.getRegionsInTransition();
if (!regionsInTransition.isEmpty()) {
- for (RegionState regionState: regionsInTransition.values()) {
+ for (RegionState regionState: regionsInTransition) {
ServerName serverName = regionState.getServerName();
if (!regionState.getRegion().isMetaRegion()
&& serverName != null && onlineServers.contains(serverName)) {
@@ -542,7 +542,7 @@ public class AssignmentManager {
}
}
}
- processRegionsInTransition(regionStates.getRegionsInTransition().values());
+ processRegionsInTransition(regionStates.getRegionsInTransition());
}
// Now we can safely claim failover cleanup completed and enable
@@ -2010,7 +2010,7 @@ public class AssignmentManager {
long oldestRITTime = 0;
int ritThreshold = this.server.getConfiguration().
getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000);
- for (RegionState state: regionStates.getRegionsInTransition().values()) {
+ for (RegionState state: regionStates.getRegionsInTransition()) {
totalRITs++;
long ritTime = currentTime - state.getStamp();
if (ritTime > ritThreshold) { // more than the threshold
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index b9abc65..c93b307 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Triple;
/**
@@ -86,7 +87,17 @@ public class CatalogJanitor extends ScheduledChore {
* @param enabled
*/
public boolean setEnabled(final boolean enabled) {
- return this.enabled.getAndSet(enabled);
+ boolean alreadyEnabled = this.enabled.getAndSet(enabled);
+ // If disabling is requested on an already enabled chore, we could have an active
+ // scan still going on, callers might not be aware of that and do further action thinkng
+ // that no action would be from this chore. In this case, the right action is to wait for
+ // the active scan to complete before exiting this function.
+ if (!enabled && alreadyEnabled) {
+ while (alreadyRunning.get()) {
+ Threads.sleepWithoutInterrupt(100);
+ }
+ }
+ return alreadyEnabled;
}
boolean getEnabled() {
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 109097b..d368ffb 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -1254,7 +1254,7 @@ public class HMaster extends HRegionServer implements MasterServices {
if (!this.loadBalancerTracker.isBalancerOn()) return false;
// Only allow one balance run at at time.
if (this.assignmentManager.getRegionStates().isRegionsInTransition()) {
- Map regionsInTransition =
+ Set regionsInTransition =
this.assignmentManager.getRegionStates().getRegionsInTransition();
// if hbase:meta region is in transition, result of assignment cannot be recorded
// ignore the force flag in that case
@@ -2152,7 +2152,7 @@ public class HMaster extends HRegionServer implements MasterServices {
String clusterId = fileSystemManager != null ?
fileSystemManager.getClusterId().toString() : null;
- Map regionsInTransition = assignmentManager != null ?
+ Set regionsInTransition = assignmentManager != null ?
assignmentManager.getRegionStates().getRegionsInTransition() : null;
String[] coprocessors = cpHost != null ? getMasterCoprocessors() : null;
boolean balancerOn = loadBalancerTracker != null ?
@@ -2378,10 +2378,6 @@ public class HMaster extends HRegionServer implements MasterServices {
return this.initializationBeforeMetaAssignment;
}
- public void assignRegion(HRegionInfo hri) {
- assignmentManager.assign(hri);
- }
-
/**
* Compute the average load across all region servers.
* Currently, this uses a very naive computation - just uses the number of
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
index fc5b474..a921ab5 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
@@ -24,6 +24,7 @@ import java.io.PrintStream;
import java.io.PrintWriter;
import java.util.Date;
import java.util.Map;
+import java.util.Set;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
@@ -116,11 +117,9 @@ public class MasterDumpServlet extends StateDumpServlet {
return;
}
- Map regionsInTransition =
- am.getRegionStates().getRegionsInTransition();
- for (Map.Entry e : regionsInTransition.entrySet()) {
- String rid = e.getKey();
- RegionState rs = e.getValue();
+ Set regionsInTransition = am.getRegionStates().getRegionsInTransition();
+ for (RegionState rs : regionsInTransition) {
+ String rid = rs.getRegion().getRegionNameAsString();
out.println("Region " + rid + ": " + rs.toDescriptiveString());
}
}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
index bc5173a..82e28df 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
@@ -144,7 +144,7 @@ public class RegionStateStore {
if (metaRegion == null) {
Configuration conf = server.getConfiguration();
// Config to determine the no of HConnections to META.
- // A single HConnection should be sufficient in most cases. Only if
+ // A single Connection should be sufficient in most cases. Only if
// you are doing lot of writes (>1M) to META,
// increasing this value might improve the write throughput.
multiHConnection =
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
index be9758a..9da8033 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@@ -28,9 +30,9 @@ import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.Collections;
-import java.util.Comparator;
+import java.util.SortedSet;
import java.util.TreeMap;
+import java.util.TreeSet;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@@ -63,6 +65,15 @@ import org.apache.hadoop.hbase.util.Pair;
public class RegionStates {
private static final Log LOG = LogFactory.getLog(RegionStates.class);
+ public final static RegionStateStampComparator REGION_STATE_COMPARATOR =
+ new RegionStateStampComparator();
+ private static class RegionStateStampComparator implements Comparator {
+ @Override
+ public int compare(RegionState l, RegionState r) {
+ return Long.compare(l.getStamp(), r.getStamp());
+ }
+ }
+
/**
* Regions currently in transition.
*/
@@ -205,31 +216,16 @@ public class RegionStates {
/**
* Get regions in transition and their states
*/
- @SuppressWarnings("unchecked")
- public synchronized Map getRegionsInTransition() {
- return (Map)regionsInTransition.clone();
+ public synchronized Set getRegionsInTransition() {
+ return new HashSet(regionsInTransition.values());
}
- @SuppressWarnings("unchecked")
- public synchronized Map getRegionsInTransitionOrderedByTimestamp() {
- Map rit = (Map)regionsInTransition.clone();
- List> list = new LinkedList<>(rit.entrySet());
-
- // Compare the RITs' timestamps for ordering.
- Comparator> c =
- new Comparator>() {
- @Override
- public int compare(Map.Entry o1, Map.Entry o2) {
- return ((Long)o1.getValue().getStamp()).compareTo((Long)o2.getValue().getStamp());
- }
- };
-
- Collections.sort(list, c);
- Map result = new LinkedHashMap<>();
- for (Map.Entry entry : list) {
- result.put(entry.getKey(), entry.getValue());
+ public synchronized SortedSet getRegionsInTransitionOrderedByTimestamp() {
+ final TreeSet rit = new TreeSet(REGION_STATE_COMPARATOR);
+ for (RegionState rs: regionsInTransition.values()) {
+ rit.add(rs);
}
- return result;
+ return rit;
}
/**
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 6a28006..f52dbdf 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -68,7 +68,7 @@ import com.google.common.collect.Sets;
*
*/
public abstract class BaseLoadBalancer implements LoadBalancer {
- private static final int MIN_SERVER_BALANCE = 2;
+ protected static final int MIN_SERVER_BALANCE = 2;
private volatile boolean stopped = false;
private static final List