diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java index 0b97c3f..5f36187 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.RetriesExhaustedException; import org.apache.hadoop.hbase.ipc.RpcClient.FailedServerException; @@ -122,14 +121,13 @@ public class CatalogTracker { * Begin active tracking by executing {@link #start()} post construction. Does * not timeout. * - * @param conf - * the {@link Configuration} from which a {@link HConnection} will be - * obtained; if problem, this connections + * @param conn + * the {@link HConnection} to use; if problem, this connection's * {@link HConnection#abort(String, Throwable)} will be called. * @throws IOException */ - public CatalogTracker(final Configuration conf) throws IOException { - this(null, conf, null); + public CatalogTracker(HConnection conn) throws IOException { + this(null, conn, null); } /** @@ -138,22 +136,16 @@ public class CatalogTracker { * Does not timeout. * @param zk If zk is null, we'll create an instance (and shut it down * when {@link #stop()} is called) else we'll use what is passed. - * @param conf + * @param conn the {@link HConnection} to use; if problem, this connection's + * {@link HConnection#abort(String, Throwable)} will be called. * @param abortable If fatal exception we'll call abort on this. May be null. * If it is we'll use the Connection associated with the passed * {@link Configuration} as our Abortable. * @throws IOException */ - public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf, - Abortable abortable) + public CatalogTracker(final ZooKeeperWatcher zk, HConnection conn, Abortable abortable) throws IOException { - this(zk, conf, HConnectionManager.getConnection(conf), abortable); - } - - public CatalogTracker(final ZooKeeperWatcher zk, final Configuration conf, - HConnection connection, Abortable abortable) - throws IOException { - this.connection = connection; + this.connection = conn; if (abortable == null) { // A connection is abortable. this.abortable = this.connection; @@ -174,8 +166,8 @@ public class CatalogTracker { if (zk == null) { // Create our own. Set flag so we tear it down on stop. this.zookeeper = - new ZooKeeperWatcher(conf, "catalogtracker-on-" + connection.toString(), - abortable); + new ZooKeeperWatcher(this.connection.getConfiguration(), + "catalogtracker-on-" + connection.toString(), abortable); instantiatedzkw = true; } else { this.zookeeper = zk; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java index a5c23bd..2c3d960 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -167,7 +168,7 @@ public class MetaReader { // HTable instance uses the CatalogTracker's connection. org.apache.hadoop.hbase.client.HConnection c = catalogTracker.getConnection(); if (c == null) throw new NullPointerException("No connection"); - return new HTable(catalogTracker.getConnection().getConfiguration(), tableName); + return (HTable) c.getTable(tableName); } /** @@ -598,17 +599,19 @@ public class MetaReader { /** * Count regions in hbase:meta for passed table. - * @param c + * @param conn * @param tableName * @return Count or regions in table tableName * @throws IOException */ - public static int getRegionCount(final Configuration c, final String tableName) throws IOException { - HTable t = new HTable(c, tableName); + public static int getRegionCount(final HConnection conn, final String tableName) + throws IOException { + HTable t = null; try { + t = (HTable) conn.getTable(tableName); return t.getRegionLocations().size(); } finally { - t.close(); + if (t != null) t.close(); } } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java index 807975e..70915a1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java @@ -79,7 +79,7 @@ public class ClientScanner extends AbstractClientScanner { */ public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName) throws IOException { - this(conf, scan, tableName, HConnectionManager.getConnection(conf)); + this(conf, scan, tableName, HConnectionManager.createConnection(conf)); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java index 77fd2aa..5074c57 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java @@ -70,7 +70,7 @@ public class ClientSmallScanner extends ClientScanner { */ public ClientSmallScanner(final Configuration conf, final Scan scan, final TableName tableName) throws IOException { - this(conf, scan, tableName, HConnectionManager.getConnection(conf)); + this(conf, scan, tableName, HConnectionManager.createConnection(conf)); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java index cef0945..d707c42 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java @@ -108,13 +108,6 @@ class ConnectionManager { public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server"; private static final String CLIENT_NONCES_ENABLED_KEY = "hbase.client.nonces.enabled"; - // An LRU Map of HConnectionKey -> HConnection (TableServer). All - // access must be synchronized. This map is not private because tests - // need to be able to tinker with it. - static final Map CONNECTION_INSTANCES; - - public static final int MAX_CACHED_CONNECTION_INSTANCES; - /** * Global nonceGenerator shared per client.Currently there's no reason to limit its scope. * Once it's set under nonceGeneratorCreateLock, it is never unset or changed. @@ -123,23 +116,6 @@ class ConnectionManager { /** The nonce generator lock. Only taken when creating HConnection, which gets a private copy. */ private static Object nonceGeneratorCreateLock = new Object(); - static { - // We set instances to one more than the value specified for {@link - // HConstants#ZOOKEEPER_MAX_CLIENT_CNXNS}. By default, the zk default max - // connections to the ensemble from the one client is 30, so in that case we - // should run into zk issues before the LRU hit this value of 31. - MAX_CACHED_CONNECTION_INSTANCES = HBaseConfiguration.create().getInt( - HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS) + 1; - CONNECTION_INSTANCES = new LinkedHashMap( - (int) (MAX_CACHED_CONNECTION_INSTANCES / 0.75F) + 1, 0.75F, true) { - @Override - protected boolean removeEldestEntry( - Map.Entry eldest) { - return size() > MAX_CACHED_CONNECTION_INSTANCES; - } - }; - } - /** Dummy nonce generator for disabled nonces. */ static class NoNonceGenerator implements NonceGenerator { @Override @@ -175,43 +151,9 @@ class ConnectionManager { } /** - * Get the connection that goes with the passed conf configuration instance. - * If no current connection exists, method creates a new connection and keys it using - * connection-specific properties from the passed {@link Configuration}; see - * {@link HConnectionKey}. - * @param conf configuration - * @return HConnection object for conf - * @throws ZooKeeperConnectionException - */ - @Deprecated - public static HConnection getConnection(final Configuration conf) throws IOException { - return getConnectionInternal(conf); - } - - - static ClusterConnection getConnectionInternal(final Configuration conf) - throws IOException { - HConnectionKey connectionKey = new HConnectionKey(conf); - synchronized (CONNECTION_INSTANCES) { - HConnectionImplementation connection = CONNECTION_INSTANCES.get(connectionKey); - if (connection == null) { - connection = (HConnectionImplementation)createConnection(conf, true); - CONNECTION_INSTANCES.put(connectionKey, connection); - } else if (connection.isClosed()) { - ConnectionManager.deleteConnection(connectionKey, true); - connection = (HConnectionImplementation)createConnection(conf, true); - CONNECTION_INSTANCES.put(connectionKey, connection); - } - connection.incCount(); - return connection; - } - } - - /** * Create a new HConnection instance using the passed conf instance. - *

Note: This bypasses the usual HConnection life cycle management done by - * {@link #getConnection(Configuration)}. The caller is responsible for - * calling {@link HConnection#close()} on the returned connection instance. + *

The caller is responsible for calling {@link HConnection#close()} on the + * returned connection instance. * * This is the recommended way to create HConnections. * {@code @@ -228,20 +170,14 @@ class ConnectionManager { * @throws ZooKeeperConnectionException */ public static HConnection createConnection(Configuration conf) throws IOException { - return createConnectionInternal(conf); - } - - static ClusterConnection createConnectionInternal(Configuration conf) throws IOException { UserProvider provider = UserProvider.instantiate(conf); - return createConnection(conf, false, null, provider.getCurrent()); + return createConnection(conf, null, provider.getCurrent()); } /** * Create a new HConnection instance using the passed conf instance. - *

Note: This bypasses the usual HConnection life cycle management done by - * {@link #getConnection(Configuration)}. The caller is responsible for - * calling {@link HConnection#close()} on the returned connection instance. - * This is the recommended way to create HConnections. + *

The caller is responsible for calling {@link HConnection#close()} on the + * returned connection instance. This is the recommended way to create HConnections. * {@code * ExecutorService pool = ...; * HConnection connection = HConnectionManager.createConnection(conf, pool); @@ -259,15 +195,13 @@ class ConnectionManager { public static HConnection createConnection(Configuration conf, ExecutorService pool) throws IOException { UserProvider provider = UserProvider.instantiate(conf); - return createConnection(conf, false, pool, provider.getCurrent()); + return createConnection(conf, pool, provider.getCurrent()); } /** * Create a new HConnection instance using the passed conf instance. - *

Note: This bypasses the usual HConnection life cycle management done by - * {@link #getConnection(Configuration)}. The caller is responsible for - * calling {@link HConnection#close()} on the returned connection instance. - * This is the recommended way to create HConnections. + *

The caller is responsible for calling {@link HConnection#close()} on the + * returned connection instance. This is the recommended way to create HConnections. * {@code * ExecutorService pool = ...; * HConnection connection = HConnectionManager.createConnection(conf, pool); @@ -284,15 +218,13 @@ class ConnectionManager { */ public static HConnection createConnection(Configuration conf, User user) throws IOException { - return createConnection(conf, false, null, user); + return createConnection(conf, null, user); } /** * Create a new HConnection instance using the passed conf instance. - *

Note: This bypasses the usual HConnection life cycle management done by - * {@link #getConnection(Configuration)}. The caller is responsible for - * calling {@link HConnection#close()} on the returned connection instance. - * This is the recommended way to create HConnections. + *

The caller is responsible for calling {@link HConnection#close()} on the + * returned connection instance. This is the recommended way to create HConnections. * {@code * ExecutorService pool = ...; * HConnection connection = HConnectionManager.createConnection(conf, pool); @@ -308,21 +240,8 @@ class ConnectionManager { * @return HConnection object for conf * @throws ZooKeeperConnectionException */ - public static HConnection createConnection(Configuration conf, ExecutorService pool, User user) - throws IOException { - return createConnection(conf, false, pool, user); - } - - @Deprecated - static HConnection createConnection(final Configuration conf, final boolean managed) - throws IOException { - UserProvider provider = UserProvider.instantiate(conf); - return createConnection(conf, managed, null, provider.getCurrent()); - } - - @Deprecated - static ClusterConnection createConnection(final Configuration conf, final boolean managed, - final ExecutorService pool, final User user) + static ClusterConnection createConnection(final Configuration conf, + final ExecutorService pool, final User user) throws IOException { String className = conf.get(HConnection.HBASE_CLIENT_CONNECTION_IMPL, HConnectionImplementation.class.getName()); @@ -335,95 +254,15 @@ class ConnectionManager { try { // Default HCM#HCI is not accessible; make it so before invoking. Constructor constructor = - clazz.getDeclaredConstructor(Configuration.class, - boolean.class, ExecutorService.class, User.class); + clazz.getDeclaredConstructor(Configuration.class, ExecutorService.class, User.class); constructor.setAccessible(true); - return (ClusterConnection) constructor.newInstance(conf, managed, pool, user); + return (ClusterConnection) constructor.newInstance(conf, pool, user); } catch (Exception e) { throw new IOException(e); } } /** - * Delete connection information for the instance specified by passed configuration. - * If there are no more references to the designated connection connection, this method will - * then close connection to the zookeeper ensemble and let go of all associated resources. - * - * @param conf configuration whose identity is used to find {@link HConnection} instance. - * @deprecated - */ - public static void deleteConnection(Configuration conf) { - deleteConnection(new HConnectionKey(conf), false); - } - - /** - * Cleanup a known stale connection. - * This will then close connection to the zookeeper ensemble and let go of all resources. - * - * @param connection - * @deprecated - */ - public static void deleteStaleConnection(HConnection connection) { - deleteConnection(connection, true); - } - - /** - * Delete information for all connections. Close or not the connection, depending on the - * staleConnection boolean and the ref count. By default, you should use it with - * staleConnection to true. - * @deprecated - */ - public static void deleteAllConnections(boolean staleConnection) { - synchronized (CONNECTION_INSTANCES) { - Set connectionKeys = new HashSet(); - connectionKeys.addAll(CONNECTION_INSTANCES.keySet()); - for (HConnectionKey connectionKey : connectionKeys) { - deleteConnection(connectionKey, staleConnection); - } - CONNECTION_INSTANCES.clear(); - } - } - - /** - * Delete information for all connections.. - * @deprecated kept for backward compatibility, but the behavior is broken. HBASE-8983 - */ - @Deprecated - public static void deleteAllConnections() { - deleteAllConnections(false); - } - - - @Deprecated - private static void deleteConnection(HConnection connection, boolean staleConnection) { - synchronized (CONNECTION_INSTANCES) { - for (Entry e: CONNECTION_INSTANCES.entrySet()) { - if (e.getValue() == connection) { - deleteConnection(e.getKey(), staleConnection); - break; - } - } - } - } - - @Deprecated - private static void deleteConnection(HConnectionKey connectionKey, boolean staleConnection) { - synchronized (CONNECTION_INSTANCES) { - HConnectionImplementation connection = CONNECTION_INSTANCES.get(connectionKey); - if (connection != null) { - connection.decCount(); - if (connection.isZeroReference() || staleConnection) { - CONNECTION_INSTANCES.remove(connectionKey); - connection.internalClose(); - } - } else { - LOG.error("Connection not found in the list, can't delete it "+ - "(connection key=" + connectionKey + "). May be the key was modified?", new Exception()); - } - } - } - - /** * It is provided for unit test cases which verify the behavior of region * location cache prefetch. * @return Number of cached regions for the table. @@ -473,7 +312,7 @@ class ConnectionManager { return null; } Configuration conf = connectable.conf; - HConnection connection = getConnection(conf); + HConnection connection = createConnection(conf); boolean connectSucceeded = false; try { T returnValue = connectable.connect(connection); @@ -555,9 +394,6 @@ class ConnectionManager { private int refCount; - // indicates whether this connection's life cycle is managed (by us) - private boolean managed; - private User user; /** @@ -565,27 +401,15 @@ class ConnectionManager { */ Registry registry; - HConnectionImplementation(Configuration conf, boolean managed) throws IOException { - this(conf, managed, null, null); - } - /** * constructor * @param conf Configuration object - * @param managed If true, does not do full shutdown on close; i.e. cleanup of connection - * to zk and shutdown of all services; we just close down the resources this connection was - * responsible for and decrement usage counters. It is up to the caller to do the full - * cleanup. It is set when we want have connection sharing going on -- reuse of zk connection, - * and cached region locations, established regionserver connections, etc. When connections - * are shared, we have reference counting going on and will only do full cleanup when no more - * users of an HConnectionImplementation instance. */ - HConnectionImplementation(Configuration conf, boolean managed, - ExecutorService pool, User user) throws IOException { + HConnectionImplementation(Configuration conf, ExecutorService pool, User user) + throws IOException { this(conf); this.user = user; this.batchPool = pool; - this.managed = managed; this.registry = setupRegistry(); retrieveClusterId(); @@ -672,9 +496,6 @@ class ConnectionManager { @Override public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException { - if (managed) { - throw new IOException("The connection has to be unmanaged."); - } return new HTable(tableName, this, pool); } @@ -883,7 +704,7 @@ class ConnectionManager { return true; } }; - MetaScanner.metaScan(conf, this, visitor, tableName); + MetaScanner.metaScan(this, visitor, tableName); return available.get() && (regionCount.get() > 0); } @@ -928,7 +749,7 @@ class ConnectionManager { return true; } }; - MetaScanner.metaScan(conf, this, visitor, tableName); + MetaScanner.metaScan(this, visitor, tableName); // +1 needs to be added so that the empty start row is also taken into account return available.get() && (regionCount.get() == splitKeys.length + 1); } @@ -969,7 +790,7 @@ class ConnectionManager { @Override public List locateRegions(final TableName tableName, final boolean useCache, final boolean offlined) throws IOException { - NavigableMap regions = MetaScanner.allTableRegions(conf, this, + NavigableMap regions = MetaScanner.allTableRegions(this, tableName, offlined); final List locations = new ArrayList(); for (HRegionInfo regionInfo : regions.keySet()) { @@ -1079,7 +900,7 @@ class ConnectionManager { }; try { // pre-fetch certain number of regions info at region cache. - MetaScanner.metaScan(conf, this, visitor, tableName, row, + MetaScanner.metaScan(this, visitor, tableName, row, this.prefetchRegionLimit, TableName.META_TABLE_NAME); } catch (IOException e) { if (ExceptionUtil.isInterrupt(e)) { @@ -2428,7 +2249,8 @@ class ConnectionManager { return refCount == 0; } - void internalClose() { + @Override + public void close() { if (this.closed) { return; } @@ -2443,19 +2265,6 @@ class ConnectionManager { } } - @Override - public void close() { - if (managed) { - if (aborted) { - ConnectionManager.deleteStaleConnection(this); - } else { - ConnectionManager.deleteConnection(this, false); - } - } else { - internalClose(); - } - } - /** * Close the connection for good, regardless of what the current value of * {@link #refCount} is. Ideally, {@link #refCount} should be zero at this diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 2dc212a..d969126 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -186,9 +186,7 @@ public class HBaseAdmin implements Abortable, Closeable { */ public HBaseAdmin(Configuration c) throws MasterNotRunningException, ZooKeeperConnectionException, IOException { - // Will not leak connections, as the new implementation of the constructor - // does not throw exceptions anymore. - this(ConnectionManager.getConnectionInternal(new Configuration(c))); + this(ConnectionManager.createConnection(new Configuration(c))); this.cleanupConnectionOnClose = true; } @@ -199,9 +197,7 @@ public class HBaseAdmin implements Abortable, Closeable { * @param connection The HConnection instance to use * @throws MasterNotRunningException, ZooKeeperConnectionException are not * thrown anymore but kept into the interface for backward api compatibility - * @deprecated Do not use this internal ctor. */ - @Deprecated public HBaseAdmin(HConnection connection) throws MasterNotRunningException, ZooKeeperConnectionException { this((ClusterConnection)connection); @@ -223,15 +219,13 @@ public class HBaseAdmin implements Abortable, Closeable { /** * @return A new CatalogTracker instance; call {@link #cleanupCatalogTracker(CatalogTracker)} * to cleanup the returned catalog tracker. - * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException - * @throws IOException * @see #cleanupCatalogTracker(CatalogTracker) */ private synchronized CatalogTracker getCatalogTracker() - throws ZooKeeperConnectionException, IOException { + throws IOException { CatalogTracker ct = null; try { - ct = new CatalogTracker(this.conf); + ct = new CatalogTracker(getConnection()); ct.start(); } catch (InterruptedException e) { // Let it out as an IOE for now until we redo all so tolerate IEs @@ -528,7 +522,7 @@ public class HBaseAdmin implements Abortable, Closeable { return true; } }; - MetaScanner.metaScan(conf, connection, visitor, desc.getTableName()); + MetaScanner.metaScan(connection, visitor, desc.getTableName()); if (actualRegCount.get() < numRegs) { if (tries == this.numRetries * this.retryLongerMultiplier - 1) { throw new RegionOfflineException("Only " + actualRegCount.get() + @@ -2044,7 +2038,7 @@ public class HBaseAdmin implements Abortable, Closeable { } }; - MetaScanner.metaScan(conf, connection, visitor, null); + MetaScanner.metaScan(connection, visitor, null); pair = result.get(); } return pair; @@ -2333,7 +2327,7 @@ public class HBaseAdmin implements Abortable, Closeable { ConnectionManager.HConnectionImplementation connection = (ConnectionManager.HConnectionImplementation) - HConnectionManager.getConnection(copyOfConf); + HConnectionManager.createConnection(copyOfConf); try { // Check ZK first. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java index 458f2ec..a2e79e1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java @@ -235,9 +235,11 @@ public interface HConnection extends Abortable, Closeable { TableName[] listTableNames() throws IOException; /** + * Connects to the master to get the table descriptor. * @param tableName table name * @return table metadata - * @throws IOException if a remote or network exception occurs + * @throws IOException if the connection to master fails or if the table + * is not found. */ HTableDescriptor getHTableDescriptor(TableName tableName) throws IOException; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java index 7436b85..188130c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -27,7 +27,6 @@ import java.net.SocketException; import java.util.ArrayList; import java.util.Date; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -51,7 +50,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Chore; -import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; @@ -113,64 +111,12 @@ import com.google.protobuf.ServiceException; * connection.close(); * } * } - *

The following logic and API will be removed in the future: - *

This class has a static Map of {@link HConnection} instances keyed by - * {@link Configuration}; all invocations of {@link #getConnection(Configuration)} - * that pass the same {@link Configuration} instance will be returned the same - * {@link HConnection} instance (Adding properties to a Configuration - * instance does not change its object identity; for more on how this is done see - * {@link HConnectionKey}). Sharing {@link HConnection} - * instances is usually what you want; all clients of the {@link HConnection} - * instances share the HConnections' cache of Region locations rather than each - * having to discover for itself the location of meta, etc. It makes - * sense for the likes of the pool of HTables class {@link HTablePool}, for - * instance (If concerned that a single {@link HConnection} is insufficient - * for sharing amongst clients in say an heavily-multithreaded environment, - * in practise its not proven to be an issue. Besides, {@link HConnection} is - * implemented atop Hadoop RPC and as of this writing, Hadoop RPC does a - * connection per cluster-member, exclusively). - * - *

But sharing connections makes clean up of {@link HConnection} instances a little awkward. - * Currently, clients cleanup by calling {@link #deleteConnection(Configuration)}. This will - * shutdown the zookeeper connection the HConnection was using and clean up all - * HConnection resources as well as stopping proxies to servers out on the - * cluster. Not running the cleanup will not end the world; it'll - * just stall the closeup some and spew some zookeeper connection failed - * messages into the log. Running the cleanup on a {@link HConnection} that is - * subsequently used by another will cause breakage so be careful running - * cleanup. - *

To create a {@link HConnection} that is not shared by others, you can - * create a new {@link Configuration} instance, pass this new instance to - * {@link #getConnection(Configuration)}, and then when done, close it up by - * doing something like the following: - *

- * {@code
- * Configuration newConfig = new Configuration(originalConf);
- * HConnection connection = HConnectionManager.getConnection(newConfig);
- * // Use the connection to your hearts' delight and then when done...
- * HConnectionManager.deleteConnection(newConfig, true);
- * }
- * 
- *

Cleanup used to be done inside in a shutdown hook. On startup we'd - * register a shutdown hook that called {@link #deleteAllConnections()} - * on its way out but the order in which shutdown hooks run is not defined so - * were problematic for clients of HConnection that wanted to register their - * own shutdown hooks so we removed ours though this shifts the onus for - * cleanup to the client. */ @SuppressWarnings("serial") @InterfaceAudience.Public @InterfaceStability.Evolving public class HConnectionManager { - @Deprecated - public static final String RETRIES_BY_SERVER_KEY = - ConnectionManager.RETRIES_BY_SERVER_KEY; - - @Deprecated - public static final int MAX_CACHED_CONNECTION_INSTANCES = - ConnectionManager.MAX_CACHED_CONNECTION_INSTANCES; - /* * Non-instantiable. */ @@ -179,23 +125,9 @@ public class HConnectionManager { } /** - * Get the connection that goes with the passed conf configuration instance. - * If no current connection exists, method creates a new connection and keys it using - * connection-specific properties from the passed {@link Configuration}; see - * {@link HConnectionKey}. - * @param conf configuration - * @return HConnection object for conf - * @throws ZooKeeperConnectionException - */ - public static HConnection getConnection(final Configuration conf) throws IOException { - return ConnectionManager.getConnectionInternal(conf); - } - - /** * Create a new HConnection instance using the passed conf instance. - *

Note: This bypasses the usual HConnection life cycle management done by - * {@link #getConnection(Configuration)}. The caller is responsible for - * calling {@link HConnection#close()} on the returned connection instance. + *

The caller is responsible for calling {@link HConnection#close()} on the + * returned connection instance. * * This is the recommended way to create HConnections. * {@code @@ -212,16 +144,14 @@ public class HConnectionManager { * @throws ZooKeeperConnectionException */ public static HConnection createConnection(Configuration conf) throws IOException { - return ConnectionManager.createConnectionInternal(conf); + return ConnectionManager.createConnection(conf); } /** * Create a new HConnection instance using the passed conf instance. - *

Note: This bypasses the usual HConnection life cycle management done by - * {@link #getConnection(Configuration)}. The caller is responsible for - * calling {@link HConnection#close()} on the returned connection instance. - * This is the recommended way to create HConnections. + *

The caller is responsible for calling {@link HConnection#close()} on the + * returned connection instance. This is the recommended way to create HConnections. * {@code * ExecutorService pool = ...; * HConnection connection = HConnectionManager.createConnection(conf, pool); @@ -243,10 +173,8 @@ public class HConnectionManager { /** * Create a new HConnection instance using the passed conf instance. - *

Note: This bypasses the usual HConnection life cycle management done by - * {@link #getConnection(Configuration)}. The caller is responsible for - * calling {@link HConnection#close()} on the returned connection instance. - * This is the recommended way to create HConnections. + *

The caller is responsible for calling {@link HConnection#close()} on the + * returned connection instance. This is the recommended way to create HConnections. * {@code * ExecutorService pool = ...; * HConnection connection = HConnectionManager.createConnection(conf, pool); @@ -268,10 +196,8 @@ public class HConnectionManager { /** * Create a new HConnection instance using the passed conf instance. - *

Note: This bypasses the usual HConnection life cycle management done by - * {@link #getConnection(Configuration)}. The caller is responsible for - * calling {@link HConnection#close()} on the returned connection instance. - * This is the recommended way to create HConnections. + *

The caller is responsible for calling {@link HConnection#close()} on the + * returned connection instance. This is the recommended way to create HConnections. * {@code * ExecutorService pool = ...; * HConnection connection = HConnectionManager.createConnection(conf, pool); @@ -292,60 +218,6 @@ public class HConnectionManager { return ConnectionManager.createConnection(conf, pool, user); } - @Deprecated - static HConnection createConnection(final Configuration conf, final boolean managed) - throws IOException { - return ConnectionManager.createConnection(conf, managed); - } - - @Deprecated - static ClusterConnection createConnection(final Configuration conf, final boolean managed, - final ExecutorService pool, final User user) throws IOException { - return ConnectionManager.createConnection(conf, managed, pool, user); - } - - /** - * Delete connection information for the instance specified by passed configuration. - * If there are no more references to the designated connection connection, this method will - * then close connection to the zookeeper ensemble and let go of all associated resources. - * - * @param conf configuration whose identity is used to find {@link HConnection} instance. - * @deprecated - */ - public static void deleteConnection(Configuration conf) { - ConnectionManager.deleteConnection(conf); - } - - /** - * Cleanup a known stale connection. - * This will then close connection to the zookeeper ensemble and let go of all resources. - * - * @param connection - * @deprecated - */ - public static void deleteStaleConnection(HConnection connection) { - ConnectionManager.deleteStaleConnection(connection); - } - - /** - * Delete information for all connections. Close or not the connection, depending on the - * staleConnection boolean and the ref count. By default, you should use it with - * staleConnection to true. - * @deprecated - */ - public static void deleteAllConnections(boolean staleConnection) { - ConnectionManager.deleteAllConnections(staleConnection); - } - - /** - * Delete information for all connections.. - * @deprecated kept for backward compatibility, but the behavior is broken. HBASE-8983 - */ - @Deprecated - public static void deleteAllConnections() { - ConnectionManager.deleteAllConnections(); - } - /** * This convenience method invokes the given {@link HConnectable#connect} * implementation using a {@link HConnection} instance that lasts just for the diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index a9b21aa..e354490 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -191,7 +191,7 @@ public class HTable implements HTableInterface { this.connection = null; return; } - this.connection = ConnectionManager.getConnectionInternal(conf); + this.connection = (ClusterConnection) ConnectionManager.createConnection(conf); this.configuration = conf; this.pool = getDefaultExecutor(conf); @@ -267,7 +267,7 @@ public class HTable implements HTableInterface { */ public HTable(Configuration conf, final TableName tableName, final ExecutorService pool) throws IOException { - this.connection = ConnectionManager.getConnectionInternal(conf); + this.connection = (ClusterConnection) ConnectionManager.createConnection(conf); this.configuration = conf; this.pool = pool; this.tableName = tableName; @@ -303,10 +303,9 @@ public class HTable implements HTableInterface { } /** - * Creates an object to access a HBase table. - * Shares zookeeper connection and other resources with other HTable instances - * created with the same connection instance. - * Visible only for HTableWrapper which is in different package. + * Creates an object to access a HBase table. Shares zookeeper connection + * and other resources with other HTable instances created with the same + * {@code connection} instance. Visible only for ClusterConnection which is in different package. * Should not be used by exernal code. * @param tableName Name of the table. * @param connection HConnection to be used. @@ -314,7 +313,7 @@ public class HTable implements HTableInterface { * @throws IOException if a remote or network exception occurs */ @InterfaceAudience.Private - public HTable(TableName tableName, final ClusterConnection connection, + protected HTable(TableName tableName, final ClusterConnection connection, final ExecutorService pool) throws IOException { if (connection == null || connection.isClosed()) { throw new IllegalArgumentException("Connection is null or closed."); @@ -331,6 +330,7 @@ public class HTable implements HTableInterface { /** * For internal testing. */ + @VisibleForTesting protected HTable() { tableName = null; cleanupPoolOnClose = false; @@ -624,7 +624,7 @@ public class HTable implements HTableInterface { */ public NavigableMap getRegionLocations() throws IOException { // TODO: Odd that this returns a Map of HRI to SN whereas getRegionLocation, singular, returns an HRegionLocation. - return MetaScanner.allTableRegions(getConfiguration(), this.connection, getName(), false); + return MetaScanner.allTableRegions(this.connection, getName(), false); } /** @@ -1671,18 +1671,4 @@ public class HTable implements HTableInterface { public String toString() { return tableName + ";" + connection; } - - /** - * Run basic test. - * @param args Pass table name and row and will get the content. - * @throws IOException - */ - public static void main(String[] args) throws IOException { - HTable t = new HTable(HBaseConfiguration.create(), args[0]); - try { - System.out.println(t.get(new Get(Bytes.toBytes(args[1])))); - } finally { - t.close(); - } - } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java index 1515b37..a4a7f8e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java @@ -26,11 +26,12 @@ import java.io.IOException; /** * Factory for creating HTable instances. - * - * @since 0.21.0 + * @deprecated Replaced by {@link HConnectionManager#createConnection(Configuration)} + * and its sibling methods. */ @InterfaceAudience.Public @InterfaceStability.Stable +@Deprecated public class HTableFactory implements HTableInterfaceFactory { @Override public HTableInterface createHTableInterface(Configuration config, diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java index 9d378ae..906fdda 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java @@ -76,24 +76,24 @@ public class HTableMultiplexer { /** The map between each region server to its flush worker */ private Map serverToFlushWorkerMap; - private Configuration conf; + private HConnection conn; private int retryNum; private int perRegionServerBufferQueueSize; /** * - * @param conf The HBaseConfiguration + * @param conn The HConnection * @param perRegionServerBufferQueueSize determines the max number of the buffered Put ops * for each region server before dropping the request. */ - public HTableMultiplexer(Configuration conf, + public HTableMultiplexer(HConnection conn, int perRegionServerBufferQueueSize) throws ZooKeeperConnectionException { - this.conf = conf; + this.conn = conn; this.serverToBufferQueueMap = new ConcurrentHashMap>(); this.serverToFlushWorkerMap = new ConcurrentHashMap(); this.tableNameToHTableMap = new ConcurrentSkipListMap(); - this.retryNum = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, + this.retryNum = this.conn.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); this.perRegionServerBufferQueueSize = perRegionServerBufferQueueSize; } @@ -203,7 +203,7 @@ public class HTableMultiplexer { synchronized (this.tableNameToHTableMap) { htable = this.tableNameToHTableMap.get(tableName); if (htable == null) { - htable = new HTable(conf, tableName); + htable = (HTable) conn.getTable(tableName); this.tableNameToHTableMap.put(tableName, htable); } } @@ -221,7 +221,7 @@ public class HTableMultiplexer { serverToBufferQueueMap.put(addr, queue); // Create the flush worker - HTableFlushWorker worker = new HTableFlushWorker(conf, addr, + HTableFlushWorker worker = new HTableFlushWorker(conn, addr, this, queue, htable); this.serverToFlushWorkerMap.put(addr, worker); @@ -406,7 +406,7 @@ public class HTableMultiplexer { private static class HTableFlushWorker implements Runnable { private HRegionLocation addr; - private Configuration conf; + private HConnection conn; private LinkedBlockingQueue queue; private HTableMultiplexer htableMultiplexer; private AtomicLong totalFailedPutCount; @@ -415,11 +415,11 @@ public class HTableMultiplexer { private AtomicLong maxLatency; private HTable htable; // For Multi - public HTableFlushWorker(Configuration conf, HRegionLocation addr, + public HTableFlushWorker(HConnection conn, HRegionLocation addr, HTableMultiplexer htableMultiplexer, LinkedBlockingQueue queue, HTable htable) { this.addr = addr; - this.conf = conf; + this.conn = conn; this.htableMultiplexer = htableMultiplexer; this.queue = queue; this.totalFailedPutCount = new AtomicLong(0); @@ -471,7 +471,7 @@ public class HTableMultiplexer { * The frequency in milliseconds for the current thread to process the corresponding * buffer queue. **/ - long frequency = conf.getLong(TABLE_MULTIPLEXER_FLUSH_FREQ_MS, 100); + long frequency = conn.getConfiguration().getLong(TABLE_MULTIPLEXER_FLUSH_FREQ_MS, 100); // initial delay try { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java index 1a6ee4c..12ffda3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java @@ -29,7 +29,6 @@ import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -55,30 +54,28 @@ public class MetaScanner { * Scans the meta table and calls a visitor on each RowResult and uses a empty * start row value as table name. * - * @param configuration conf + * @param conn Connection to use. * @param visitor A custom visitor * @throws IOException e */ - public static void metaScan(Configuration configuration, - MetaScannerVisitor visitor) + public static void metaScan(HConnection conn, MetaScannerVisitor visitor) throws IOException { - metaScan(configuration, visitor, null, null, Integer.MAX_VALUE); + metaScan(conn, visitor, null, null, Integer.MAX_VALUE, TableName.META_TABLE_NAME); } /** * Scans the meta table and calls a visitor on each RowResult. Uses a table * name to locate meta regions. * - * @param configuration config - * @param connection connection to use internally (null to use a new instance) + * @param connection connection to use * @param visitor visitor object * @param userTableName User table name in meta table to start scan at. Pass * null if not interested in a particular table. * @throws IOException e */ - public static void metaScan(Configuration configuration, ClusterConnection connection, - MetaScannerVisitor visitor, TableName userTableName) throws IOException { - metaScan(configuration, connection, visitor, userTableName, null, Integer.MAX_VALUE, + public static void metaScan(HConnection connection, MetaScannerVisitor visitor, + TableName userTableName) throws IOException { + metaScan(connection, visitor, userTableName, null, Integer.MAX_VALUE, TableName.META_TABLE_NAME); } @@ -87,31 +84,7 @@ public class MetaScanner { * name and a row name to locate meta regions. And it only scans at most * rowLimit of rows. * - * @param configuration HBase configuration. - * @param visitor Visitor object. - * @param userTableName User table name in meta table to start scan at. Pass - * null if not interested in a particular table. - * @param row Name of the row at the user table. The scan will start from - * the region row where the row resides. - * @param rowLimit Max of processed rows. If it is less than 0, it - * will be set to default value Integer.MAX_VALUE. - * @throws IOException e - */ - public static void metaScan(Configuration configuration, - MetaScannerVisitor visitor, TableName userTableName, byte[] row, - int rowLimit) - throws IOException { - metaScan(configuration, null, visitor, userTableName, row, rowLimit, - TableName.META_TABLE_NAME); - } - - /** - * Scans the meta table and calls a visitor on each RowResult. Uses a table - * name and a row name to locate meta regions. And it only scans at most - * rowLimit of rows. - * - * @param configuration HBase configuration. - * @param connection connection to use internally (null to use a new instance) + * @param conn connection to use. * @param visitor Visitor object. Closes the visitor before returning. * @param tableName User table name in meta table to start scan at. Pass * null if not interested in a particular table. @@ -122,17 +95,12 @@ public class MetaScanner { * @param metaTableName Meta table to scan, root or meta. * @throws IOException e */ - static void metaScan(Configuration configuration, ClusterConnection connection, - final MetaScannerVisitor visitor, final TableName tableName, - final byte[] row, final int rowLimit, final TableName metaTableName) + public static void metaScan(HConnection conn, final MetaScannerVisitor visitor, + final TableName tableName, final byte[] row, final int rowLimit, + final TableName metaTableName) throws IOException { int rowUpperLimit = rowLimit > 0 ? rowLimit: Integer.MAX_VALUE; - HTable metaTable; - if (connection == null) { - metaTable = new HTable(configuration, TableName.META_TABLE_NAME, null); - } else { - metaTable = new HTable(TableName.META_TABLE_NAME, connection, null); - } + HTable metaTable = (HTable) conn.getTable(metaTableName); // Calculate startrow for scan. byte[] startRow; ResultScanner scanner = null; @@ -162,7 +130,7 @@ public class MetaScanner { HConstants.ZEROES, false); } final Scan scan = new Scan(startRow).addFamily(HConstants.CATALOG_FAMILY); - int scannerCaching = configuration.getInt(HConstants.HBASE_META_SCANNER_CACHING, + int scannerCaching = conn.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING, HConstants.DEFAULT_HBASE_META_SCANNER_CACHING); if (rowUpperLimit <= scannerCaching) { scan.setSmall(true); @@ -224,13 +192,13 @@ public class MetaScanner { * Used in tests. * * Lists all of the regions currently in META. - * @param conf + * @param conn Connection to use. * @param offlined True if we are to include offlined regions, false and we'll * leave out offlined regions from returned list. * @return List of all user-space regions. * @throws IOException */ - public static List listAllRegions(Configuration conf, final boolean offlined) + public static List listAllRegions(HConnection conn, final boolean offlined) throws IOException { final List regions = new ArrayList(); MetaScannerVisitor visitor = new DefaultMetaScannerVisitor() { @@ -252,21 +220,19 @@ public class MetaScanner { return true; } }; - metaScan(conf, visitor); + metaScan(conn, visitor); return regions; } /** * Lists all of the table regions currently in META. - * @param conf - * @param offlined True if we are to include offlined regions, false and we'll - * leave out offlined regions from returned list. + * @param conn cluster to use. * @return Map of all user-space regions to servers * @throws IOException */ - public static NavigableMap allTableRegions(Configuration conf, - ClusterConnection connection, final TableName tableName, - final boolean offlined) throws IOException { + public static NavigableMap allTableRegions(HConnection conn, + final TableName tableName, final boolean offlined) throws IOException { + // TODO: offlined is not respected! final NavigableMap regions = new TreeMap(); MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) { @@ -278,7 +244,7 @@ public class MetaScanner { return true; } }; - metaScan(conf, connection, visitor, tableName); + metaScan(conn, visitor, tableName); return regions; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java index 366d176..c33e854 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java @@ -29,6 +29,7 @@ import java.util.NavigableSet; import java.util.TreeMap; import java.util.concurrent.atomic.AtomicLong; +import com.sun.swing.internal.plaf.metal.resources.metal; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -36,7 +37,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; @@ -76,14 +79,13 @@ import com.google.protobuf.Message; public class AggregationClient { private static final Log log = LogFactory.getLog(AggregationClient.class); - Configuration conf; + HConnection conn; /** * Constructor with Conf object - * @param cfg */ - public AggregationClient(Configuration cfg) { - this.conf = cfg; + public AggregationClient(HConnection conn) { + this.conn = conn; } /** @@ -101,9 +103,9 @@ public class AggregationClient { public R max( final TableName tableName, final ColumnInterpreter ci, final Scan scan) throws Throwable { - HTable table = null; + HTableInterface table = null; try { - table = new HTable(conf, tableName); + table = conn.getTable(tableName); return max(table, ci, scan); } finally { if (table != null) { @@ -125,7 +127,7 @@ public class AggregationClient { * & propagated to it. */ public - R max(final HTable table, final ColumnInterpreter ci, + R max(final HTableInterface table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); class MaxCallBack implements Batch.Callback { @@ -196,9 +198,9 @@ public class AggregationClient { public R min( final TableName tableName, final ColumnInterpreter ci, final Scan scan) throws Throwable { - HTable table = null; + HTableInterface table = null; try { - table = new HTable(conf, tableName); + table = conn.getTable(tableName); return min(table, ci, scan); } finally { if (table != null) { @@ -218,7 +220,7 @@ public class AggregationClient { * @throws Throwable */ public - R min(final HTable table, final ColumnInterpreter ci, + R min(final HTableInterface table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); class MinCallBack implements Batch.Callback { @@ -276,9 +278,9 @@ public class AggregationClient { public long rowCount( final TableName tableName, final ColumnInterpreter ci, final Scan scan) throws Throwable { - HTable table = null; + HTableInterface table = null; try { - table = new HTable(conf, tableName); + table = conn.getTable(tableName); return rowCount(table, ci, scan); } finally { if (table != null) { @@ -301,7 +303,7 @@ public class AggregationClient { * @throws Throwable */ public - long rowCount(final HTable table, + long rowCount(final HTableInterface table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, true); class RowNumCallback implements Batch.Callback { @@ -350,9 +352,9 @@ public class AggregationClient { public S sum( final TableName tableName, final ColumnInterpreter ci, final Scan scan) throws Throwable { - HTable table = null; + HTableInterface table = null; try { - table = new HTable(conf, tableName); + table = conn.getTable(tableName); return sum(table, ci, scan); } finally { if (table != null) { @@ -371,7 +373,7 @@ public class AggregationClient { * @throws Throwable */ public - S sum(final HTable table, final ColumnInterpreter ci, + S sum(final HTableInterface table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); @@ -423,9 +425,9 @@ public class AggregationClient { private Pair getAvgArgs( final TableName tableName, final ColumnInterpreter ci, final Scan scan) throws Throwable { - HTable table = null; + HTableInterface table = null; try { - table = new HTable(conf, tableName); + table = conn.getTable(tableName); return getAvgArgs(table, ci, scan); } finally { if (table != null) { @@ -443,7 +445,7 @@ public class AggregationClient { * @throws Throwable */ private - Pair getAvgArgs(final HTable table, + Pair getAvgArgs(final HTableInterface table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); class AvgCallBack implements Batch.Callback> { @@ -540,7 +542,7 @@ public class AggregationClient { * @throws Throwable */ private - Pair, Long> getStdArgs(final HTable table, + Pair, Long> getStdArgs(final HTableInterface table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); class StdCallback implements Batch.Callback, Long>> { @@ -614,9 +616,9 @@ public class AggregationClient { public double std(final TableName tableName, ColumnInterpreter ci, Scan scan) throws Throwable { - HTable table = null; + HTableInterface table = null; try { - table = new HTable(conf, tableName); + table = conn.getTable(tableName); return std(table, ci, scan); } finally { if (table != null) { @@ -638,7 +640,8 @@ public class AggregationClient { * @throws Throwable */ public double std( - final HTable table, ColumnInterpreter ci, Scan scan) throws Throwable { + final HTableInterface table, + ColumnInterpreter ci, Scan scan) throws Throwable { Pair, Long> p = getStdArgs(table, ci, scan); double res = 0d; double avg = ci.divideForAvg(p.getFirst().get(0), p.getSecond()); @@ -662,7 +665,7 @@ public class AggregationClient { */ private Pair>, List> - getMedianArgs(final HTable table, + getMedianArgs(final HTableInterface table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); final NavigableMap> map = @@ -727,9 +730,9 @@ public class AggregationClient { public R median(final TableName tableName, ColumnInterpreter ci, Scan scan) throws Throwable { - HTable table = null; + HTableInterface table = null; try { - table = new HTable(conf, tableName); + table = conn.getTable(tableName); return median(table, ci, scan); } finally { if (table != null) { @@ -749,7 +752,7 @@ public class AggregationClient { * @throws Throwable */ public - R median(final HTable table, ColumnInterpreter ci, + R median(final HTableInterface table, ColumnInterpreter ci, Scan scan) throws Throwable { Pair>, List> p = getMedianArgs(table, ci, scan); byte[] startRow = null; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java index 72e3fbb..205a57a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java @@ -95,7 +95,7 @@ public class ReplicationAdmin implements Closeable { throw new RuntimeException("hbase.replication isn't true, please " + "enable it in order to use replication"); } - this.connection = HConnectionManager.getConnection(conf); + this.connection = HConnectionManager.createConnection(conf); ZooKeeperWatcher zkw = createZooKeeperWatcher(); try { this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java index e19c23c..66f7e59 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java @@ -23,10 +23,10 @@ import java.util.Map; import com.google.protobuf.HBaseZeroCopyByteString; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; @@ -49,7 +49,7 @@ import com.google.protobuf.ByteString; public class AccessControlClient { /** * Grants permission on the specified table for the specified user - * @param conf + * @param conn * @param tableName * @param userName * @param family @@ -58,14 +58,14 @@ public class AccessControlClient { * @return GrantResponse * @throws Throwable */ - public static GrantResponse grant(Configuration conf, final TableName tableName, + public static GrantResponse grant(HConnection conn, final TableName tableName, final String userName, final byte[] family, final byte[] qual, final AccessControlProtos.Permission.Action... actions) throws Throwable { HTable ht = null; try { TableName aclTableName = TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl"); - ht = new HTable(conf, aclTableName.getName()); + ht = (HTable) conn.getTable(aclTableName); Batch.Call callable = new Batch.Call() { ServerRpcController controller = new ServerRpcController(); @@ -114,7 +114,7 @@ public class AccessControlClient { /** * Revokes the permission on the table - * @param conf + * @param conn * @param username * @param tableName * @param family @@ -123,14 +123,14 @@ public class AccessControlClient { * @return RevokeResponse * @throws Throwable */ - public static RevokeResponse revoke(Configuration conf, final String username, + public static RevokeResponse revoke(HConnection conn, final String username, final TableName tableName, final byte[] family, final byte[] qualifier, final AccessControlProtos.Permission.Action... actions) throws Throwable { HTable ht = null; try { TableName aclTableName = TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl"); - ht = new HTable(conf, aclTableName.getName()); + ht = (HTable) conn.getTable(aclTableName); Batch.Call callable = new Batch.Call() { ServerRpcController controller = new ServerRpcController(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java index bf1b01f..fbe4de6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java @@ -25,9 +25,9 @@ import java.util.Map; import com.google.protobuf.HBaseZeroCopyByteString; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; import org.apache.hadoop.hbase.ipc.ServerRpcController; @@ -52,30 +52,30 @@ public class VisibilityClient { /** * Utility method for adding label to the system. * - * @param conf + * @param conn * @param label * @return VisibilityLabelsResponse * @throws Throwable */ - public static VisibilityLabelsResponse addLabel(Configuration conf, final String label) + public static VisibilityLabelsResponse addLabel(HConnection conn, final String label) throws Throwable { - return addLabels(conf, new String[] { label }); + return addLabels(conn, new String[] { label }); } /** * Utility method for adding labels to the system. * - * @param conf + * @param conn * @param labels * @return VisibilityLabelsResponse * @throws Throwable */ - public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels) + public static VisibilityLabelsResponse addLabels(HConnection conn, final String[] labels) throws Throwable { - HTable ht = null; + HTableInterface ht = null; try { - ht = new HTable(conf, LABELS_TABLE_NAME.getName()); - Batch.Call callable = + ht = conn.getTable(LABELS_TABLE_NAME); + Batch.Call callable = new Batch.Call() { ServerRpcController controller = new ServerRpcController(); BlockingRpcCallback rpcCallback = @@ -108,27 +108,27 @@ public class VisibilityClient { /** * Sets given labels globally authorized for the user. - * @param conf + * @param conn * @param auths * @param user * @return VisibilityLabelsResponse * @throws Throwable */ - public static VisibilityLabelsResponse setAuths(Configuration conf, final String[] auths, + public static VisibilityLabelsResponse setAuths(HConnection conn, final String[] auths, final String user) throws Throwable { - return setOrClearAuths(conf, auths, user, true); + return setOrClearAuths(conn, auths, user, true); } /** - * @param conf + * @param conn * @param user * @return labels, the given user is globally authorized for. * @throws Throwable */ - public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable { - HTable ht = null; + public static GetAuthsResponse getAuths(HConnection conn, final String user) throws Throwable { + HTableInterface ht = null; try { - ht = new HTable(conf, LABELS_TABLE_NAME.getName()); + ht = conn.getTable(LABELS_TABLE_NAME); Batch.Call callable = new Batch.Call() { ServerRpcController controller = new ServerRpcController(); @@ -155,22 +155,22 @@ public class VisibilityClient { /** * Removes given labels from user's globally authorized list of labels. - * @param conf + * @param conn * @param auths * @param user * @return VisibilityLabelsResponse * @throws Throwable */ - public static VisibilityLabelsResponse clearAuths(Configuration conf, final String[] auths, + public static VisibilityLabelsResponse clearAuths(HConnection conn, final String[] auths, final String user) throws Throwable { - return setOrClearAuths(conf, auths, user, false); + return setOrClearAuths(conn, auths, user, false); } - private static VisibilityLabelsResponse setOrClearAuths(Configuration conf, final String[] auths, + private static VisibilityLabelsResponse setOrClearAuths(HConnection conn, final String[] auths, final String user, final boolean setOrClear) throws IOException, ServiceException, Throwable { - HTable ht = null; + HTableInterface ht = null; try { - ht = new HTable(conf, LABELS_TABLE_NAME.getName()); + ht = conn.getTable(LABELS_TABLE_NAME); Batch.Call callable = new Batch.Call() { ServerRpcController controller = new ServerRpcController(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index 2067e01..582ca10 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -207,7 +207,13 @@ public class TestClientNoCluster extends Configured implements Tool { public void testDoNotRetryMetaScanner() throws IOException { this.conf.set("hbase.client.connection.impl", RegionServerStoppedOnScannerOpenConnection.class.getName()); - MetaScanner.metaScan(this.conf, null); + ClusterConnection conn = null; + try { + conn = (ClusterConnection) HConnectionManager.createConnection(this.conf); + MetaScanner.metaScan(conn, null); + } finally { + if (conn != null) conn.close(); + } } @Test @@ -258,8 +264,8 @@ public class TestClientNoCluster extends Configured implements Tool { final ClientService.BlockingInterface stub; ScanOpenNextThenExceptionThenRecoverConnection(Configuration conf, - boolean managed, ExecutorService pool) throws IOException { - super(conf, managed); + ExecutorService pool) throws IOException { + super(conf); // Mock up my stub so open scanner returns a scanner id and then on next, we throw // exceptions for three times and then after that, we return no more to scan. this.stub = Mockito.mock(ClientService.BlockingInterface.class); @@ -289,9 +295,9 @@ public class TestClientNoCluster extends Configured implements Tool { extends ConnectionManager.HConnectionImplementation { final ClientService.BlockingInterface stub; - RegionServerStoppedOnScannerOpenConnection(Configuration conf, boolean managed, + RegionServerStoppedOnScannerOpenConnection(Configuration conf, ExecutorService pool, User user) throws IOException { - super(conf, managed); + super(conf, pool, user); // Mock up my stub so open scanner returns a scanner id and then on next, we throw // exceptions for three times and then after that, we return no more to scan. this.stub = Mockito.mock(ClientService.BlockingInterface.class); @@ -321,9 +327,9 @@ public class TestClientNoCluster extends Configured implements Tool { extends ConnectionManager.HConnectionImplementation { final ClientService.BlockingInterface stub; - RpcTimeoutConnection(Configuration conf, boolean managed, ExecutorService pool, User user) + RpcTimeoutConnection(Configuration conf, ExecutorService pool, User user) throws IOException { - super(conf, managed); + super(conf, pool, user); // Mock up my stub so an exists call -- which turns into a get -- throws an exception this.stub = Mockito.mock(ClientService.BlockingInterface.class); try { @@ -356,10 +362,9 @@ public class TestClientNoCluster extends Configured implements Tool { final AtomicLong sequenceids = new AtomicLong(0); private final Configuration conf; - ManyServersManyRegionsConnection(Configuration conf, boolean managed, - ExecutorService pool, User user) + ManyServersManyRegionsConnection(Configuration conf, ExecutorService pool, User user) throws IOException { - super(conf, managed, pool, user); + super(conf, pool, user); int serverCount = conf.getInt("hbase.test.servers", 10); this.serversByClient = new HashMap(serverCount); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java index cda5424..9df3c2d 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.util; +import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; import org.apache.hadoop.classification.InterfaceAudience; @@ -47,4 +48,21 @@ public class ReflectionUtils { "Unable to find suitable constructor for class " + className, e); } } + + /** + * Like {@link Class#getDeclaredField(String)}, but walks the inheritance graph. + */ + public static Field getAllDeclaredField(Class clazz, String fieldName) + throws NoSuchFieldException { + try { + return clazz.getDeclaredField(fieldName); + } catch (NoSuchFieldException e) { + Class parent = clazz.getSuperclass(); + if (parent != null) { + return getAllDeclaredField(parent, fieldName); + } else { + throw e; + } + } + } } diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java index d5e5f9c..ebe693a 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java @@ -32,7 +32,7 @@ import org.apache.hadoop.fs.Path; /** * Common helpers for testing HBase that do not depend on specific server/etc. things. - * @see {@link HBaseTestingUtility} + * @see {@link org.apache.hadoop.hbase.HBaseCommonTestingUtility} * */ @InterfaceAudience.Public @@ -73,10 +73,9 @@ public class HBaseCommonTestingUtility { private File dataTestDir = null; /** - * @return Where to write test data on local filesystem, specific to - * the test. Useful for tests that do not use a cluster. - * Creates it if it does not exist already. - * @see #getTestFileSystem() + * @return Where to write test data on local filesystem, specific to the + * test. Useful for tests that do not use a cluster. Creates it if it does + * not exist already. */ public Path getDataTestDir() { if (this.dataTestDir == null){ @@ -166,7 +165,6 @@ public class HBaseCommonTestingUtility { * Should not be used by the unit tests, hence its's private. * Unit test will use a subdirectory of this directory. * @see #setupDataTestDir() - * @see #getTestFileSystem() */ private Path getBaseTestDir() { String PathName = System.getProperty( diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java index f1e0b16..13612a5 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java @@ -45,6 +45,7 @@ import com.google.common.collect.Sets; @InterfaceAudience.Private public class DistributedHBaseCluster extends HBaseCluster { + private HConnection conn; private HBaseAdmin admin; private ClusterManager clusterManager; @@ -53,7 +54,8 @@ public class DistributedHBaseCluster extends HBaseCluster { throws IOException { super(conf); this.clusterManager = clusterManager; - this.admin = new HBaseAdmin(conf); + this.conn = HConnectionManager.createConnection(conf); + this.admin = new HBaseAdmin(conn); this.initialClusterStatus = getClusterStatus(); } @@ -81,9 +83,8 @@ public class DistributedHBaseCluster extends HBaseCluster { @Override public void close() throws IOException { - if (this.admin != null) { - admin.close(); - } + if (this.admin != null) admin.close(); + if (this.conn != null) conn.close(); } @Override @@ -138,7 +139,6 @@ public class DistributedHBaseCluster extends HBaseCluster { @Override public MasterService.BlockingInterface getMaster() throws IOException { - HConnection conn = HConnectionManager.getConnection(conf); return conn.getMaster(); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithVisibilityLabels.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithVisibilityLabels.java index 1dafbaa..3a03c25 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithVisibilityLabels.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithVisibilityLabels.java @@ -114,8 +114,8 @@ public class IntegrationTestIngestWithVisibilityLabels extends IntegrationTestIn private void addLabels() throws Exception { try { - VisibilityClient.addLabels(util.getConfiguration(), LABELS); - VisibilityClient.setAuths(util.getConfiguration(), LABELS, User.getCurrent().getName()); + VisibilityClient.addLabels(util.getConnection(), LABELS); + VisibilityClient.setAuths(util.getConnection(), LABELS, User.getCurrent().getName()); } catch (Throwable t) { throw new IOException(t); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java index fb1f3e4..2275dc5 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java @@ -705,7 +705,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { if (!success) { Configuration conf = job.getConfiguration(); - HConnection conn = HConnectionManager.getConnection(conf); + HConnection conn = HConnectionManager.createConnection(conf); TableName tableName = getTableName(conf); CounterGroup g = counters.getGroup("undef"); Iterator it = g.iterator(); @@ -723,6 +723,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { HRegionLocation loc = conn.relocateRegion(tableName, key); LOG.error("unreferred row " + keyString + ", " + loc); } + conn.close(); } return success; } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java index 7da40a1..1b73357 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java @@ -122,11 +122,11 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT private void addLabelsAndAuths() throws Exception { try { - VisibilityClient.addLabels(util.getConfiguration(), LABELS); - VisibilityClient.setAuths(util.getConfiguration(), LABELS, User.getCurrent().getName()); - VisibilityClient.setAuths(util.getConfiguration(), new String[] { CONFIDENTIAL, TOPSECRET, + VisibilityClient.addLabels(util.getConnection(), LABELS); + VisibilityClient.setAuths(util.getConnection(), LABELS, User.getCurrent().getName()); + VisibilityClient.setAuths(util.getConnection(), new String[] { CONFIDENTIAL, TOPSECRET, SECRET, PRIVATE }, ADMIN.getName()); - VisibilityClient.setAuths(util.getConfiguration(), new String[] { PUBLIC }, + VisibilityClient.setAuths(util.getConnection(), new String[] { PUBLIC }, NORMAL_USER.getName()); } catch (Throwable t) { throw new IOException(t); diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index 00dd0e6..01ff6e00 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -207,7 +207,7 @@ listener - org.apache.hadoop.hbase.ServerResourceCheckerJUnitListener + org.apache.hadoop.hbase.ResourceCheckerJUnitListener diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon index e0617da..5449ce3 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon @@ -43,7 +43,6 @@ org.apache.hadoop.hbase.NamespaceDescriptor; org.apache.hadoop.hbase.ServerLoad; org.apache.hadoop.hbase.ServerName; org.apache.hadoop.hbase.client.HBaseAdmin; -org.apache.hadoop.hbase.client.HConnectionManager; org.apache.hadoop.hbase.HTableDescriptor; org.apache.hadoop.hbase.HBaseConfiguration; org.apache.hadoop.hbase.TableName; @@ -417,7 +416,4 @@ AssignmentManager assignmentManager = master.getAssignmentManager(); -<%java> - HConnectionManager.deleteConnection(admin.getConfiguration()); - diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java index 536bbf0..24ff656 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java @@ -75,8 +75,8 @@ public class CoprocessorHConnection implements ClusterConnection { */ public static ClusterConnection getConnectionForEnvironment(CoprocessorEnvironment env) throws IOException { - ClusterConnection connection = - ConnectionManager.createConnectionInternal(env.getConfiguration()); + ClusterConnection connection = (ClusterConnection) + ConnectionManager.createConnection(env.getConfiguration()); // this bit is a little hacky - just trying to get it going for the moment if (env instanceof RegionCoprocessorEnvironment) { RegionCoprocessorEnvironment e = (RegionCoprocessorEnvironment) env; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java index 0d604e5..2d2fc3d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java @@ -85,7 +85,8 @@ public class HTableWrapper implements HTableInterface { ClusterConnection connection, ExecutorService pool) throws IOException { this.tableName = tableName; - this.table = new HTable(tableName, connection, pool); + // cast is safe because we're consuming a ClusterConnection, not HConnection. + this.table = (HTable) connection.getTable(tableName, pool); this.connection = connection; this.openTables = openTables; this.openTables.add(this); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java index 2823415..a013f49 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java @@ -25,6 +25,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.MutationSerialization; @@ -170,9 +172,16 @@ public class TableMapReduceUtil { MutationSerialization.class.getName(), ResultSerialization.class.getName()); if (partitioner == HRegionPartitioner.class) { job.setPartitionerClass(HRegionPartitioner.class); - int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table); - if (job.getNumReduceTasks() > regions) { - job.setNumReduceTasks(regions); + HConnection conn = null; + try { + // TODO: expose this HConnection instance to the caller + conn = HConnectionManager.createConnection(HBaseConfiguration.create(job)); + int regions = MetaReader.getRegionCount(conn, table); + if (job.getNumReduceTasks() > regions) { + job.setNumReduceTasks(regions); + } + } finally { + if (conn != null) conn.close(); } } else if (partitioner != null) { job.setPartitionerClass(partitioner); @@ -230,13 +239,14 @@ public class TableMapReduceUtil { * Ensures that the given number of reduce tasks for the given job * configuration does not exceed the number of regions for the given table. * + * @param conn A connection to the target HBase cluster. * @param table The table to get the region count for. * @param job The current job configuration to adjust. * @throws IOException When retrieving the table details fails. */ - public static void limitNumReduceTasks(String table, JobConf job) + public static void limitNumReduceTasks(HConnection conn, String table, JobConf job) throws IOException { - int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table); + int regions = MetaReader.getRegionCount(conn, table); if (job.getNumReduceTasks() > regions) job.setNumReduceTasks(regions); } @@ -245,13 +255,14 @@ public class TableMapReduceUtil { * Ensures that the given number of map tasks for the given job * configuration does not exceed the number of regions for the given table. * + * @param conn A connection to the target HBase cluster. * @param table The table to get the region count for. * @param job The current job configuration to adjust. * @throws IOException When retrieving the table details fails. */ - public static void limitNumMapTasks(String table, JobConf job) + public static void limitNumMapTasks(HConnection conn, String table, JobConf job) throws IOException { - int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table); + int regions = MetaReader.getRegionCount(conn, table); if (job.getNumMapTasks() > regions) job.setNumMapTasks(regions); } @@ -260,26 +271,28 @@ public class TableMapReduceUtil { * Sets the number of reduce tasks for the given job configuration to the * number of regions the given table has. * + * @param conn A connection to the target HBase cluster. * @param table The table to get the region count for. * @param job The current job configuration to adjust. * @throws IOException When retrieving the table details fails. */ - public static void setNumReduceTasks(String table, JobConf job) + public static void setNumReduceTasks(HConnection conn, String table, JobConf job) throws IOException { - job.setNumReduceTasks(MetaReader.getRegionCount(HBaseConfiguration.create(job), table)); + job.setNumReduceTasks(MetaReader.getRegionCount(conn, table)); } /** * Sets the number of map tasks for the given job configuration to the * number of regions the given table has. * + * @param conn A connection to the target HBase cluster. * @param table The table to get the region count for. * @param job The current job configuration to adjust. * @throws IOException When retrieving the table details fails. */ - public static void setNumMapTasks(String table, JobConf job) + public static void setNumMapTasks(HConnection conn, String table, JobConf job) throws IOException { - job.setNumMapTasks(MetaReader.getRegionCount(HBaseConfiguration.create(job), table)); + job.setNumMapTasks(MetaReader.getRegionCount(conn, table)); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java index a5d7c59..4ad90b2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java @@ -43,6 +43,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -593,9 +595,17 @@ public class TableMapReduceUtil { job.setOutputValueClass(Writable.class); if (partitioner == HRegionPartitioner.class) { job.setPartitionerClass(HRegionPartitioner.class); - int regions = MetaReader.getRegionCount(conf, table); - if (job.getNumReduceTasks() > regions) { - job.setNumReduceTasks(regions); + HConnection conn = null; + try { + // TODO: expose this HConnection instance to the caller + conn = HConnectionManager.createConnection( + HBaseConfiguration.create(job.getConfiguration())); + int regions = MetaReader.getRegionCount(conn, table); + if (job.getNumReduceTasks() > regions) { + job.setNumReduceTasks(regions); + } + } finally { + if (conn != null) conn.close(); } } else if (partitioner != null) { job.setPartitionerClass(partitioner); @@ -612,13 +622,14 @@ public class TableMapReduceUtil { * Ensures that the given number of reduce tasks for the given job * configuration does not exceed the number of regions for the given table. * + * @param conn A connection to the target HBase cluster. * @param table The table to get the region count for. * @param job The current job to adjust. * @throws IOException When retrieving the table details fails. */ - public static void limitNumReduceTasks(String table, Job job) + public static void limitNumReduceTasks(HConnection conn, String table, Job job) throws IOException { - int regions = MetaReader.getRegionCount(job.getConfiguration(), table); + int regions = MetaReader.getRegionCount(conn, table); if (job.getNumReduceTasks() > regions) job.setNumReduceTasks(regions); } @@ -627,13 +638,14 @@ public class TableMapReduceUtil { * Sets the number of reduce tasks for the given job configuration to the * number of regions the given table has. * + * @param conn A connection to the target HBase cluster. * @param table The table to get the region count for. * @param job The current job to adjust. * @throws IOException When retrieving the table details fails. */ - public static void setNumReduceTasks(String table, Job job) + public static void setNumReduceTasks(HConnection conn, String table, Job job) throws IOException { - job.setNumReduceTasks(MetaReader.getRegionCount(job.getConfiguration(), table)); + job.setNumReduceTasks(MetaReader.getRegionCount(conn, table)); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java index 59bc01e..8dc798c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java @@ -164,7 +164,7 @@ public class CatalogJanitor extends Chore { // Run full scan of hbase:meta catalog table passing in our custom visitor with // the start row - MetaScanner.metaScan(server.getConfiguration(), null, visitor, tableName); + MetaScanner.metaScan(server.getCatalogTracker().getConnection(), visitor, tableName); return new Triple, Map>( count.get(), mergedRegions, splitParents); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index eae4040..cdd535b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -72,6 +72,8 @@ import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.catalog.MetaReader; import org.apache.hadoop.hbase.client.ConnectionUtils; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase; @@ -681,7 +683,9 @@ MasterServices, Server { */ void initializeZKBasedSystemTrackers() throws IOException, InterruptedException, KeeperException { - this.catalogTracker = createCatalogTracker(this.zooKeeper, this.conf, this); + // Okay to create and abandon an HConnection instance because CatalogTracker#stop() closes it. + this.catalogTracker = createCatalogTracker(this.zooKeeper, + HConnectionManager.createConnection(conf), this); this.catalogTracker.start(); this.balancer = LoadBalancerFactory.getLoadBalancer(conf); @@ -723,7 +727,7 @@ MasterServices, Server { * In its own method so can intercept and mock it over in tests. * @param zk If zk is null, we'll create an instance (and shut it down * when {@link #stop(String)} is called) else we'll use what is passed. - * @param conf + * @param conn * @param abortable If fatal exception we'll call abort on this. May be null. * If it is we'll use the Connection associated with the passed * {@link Configuration} as our {@link Abortable}. @@ -731,9 +735,9 @@ MasterServices, Server { * @throws IOException */ CatalogTracker createCatalogTracker(final ZooKeeperWatcher zk, - final Configuration conf, Abortable abortable) + HConnection conn, Abortable abortable) throws IOException { - return new CatalogTracker(zk, conf, abortable); + return new CatalogTracker(zk, conn, abortable); } // Check if we should stop every 100ms @@ -2031,7 +2035,8 @@ MasterServices, Server { } }; - MetaScanner.metaScan(conf, visitor, tableName, rowKey, 1); + MetaScanner.metaScan(this.catalogTracker.getConnection(), visitor, tableName, rowKey, + 1, TableName.META_TABLE_NAME); return result.get(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java index 848ce7a..146aec4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper; import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -90,6 +91,7 @@ public class RegionPlacementMaintainer { private static boolean USE_MUNKRES_FOR_PLACING_SECONDARY_AND_TERTIARY = false; private Configuration conf; + private HConnection conn; private final boolean enforceLocality; private final boolean enforceMinAssignmentMove; private HBaseAdmin admin; @@ -123,13 +125,20 @@ public class RegionPlacementMaintainer { } } + private HConnection getConnection() throws IOException { + if (this.conn == null) { + this.conn = HConnectionManager.createConnection(this.conf); + } + return this.conn; + } + /** * @return the cached HBaseAdmin * @throws IOException */ private HBaseAdmin getHBaseAdmin() throws IOException { if (this.admin == null) { - this.admin = new HBaseAdmin(this.conf); + this.admin = new HBaseAdmin(getConnection()); } return this.admin; } @@ -141,7 +150,7 @@ public class RegionPlacementMaintainer { public SnapshotOfRegionAssignmentFromMeta getRegionAssignmentSnapshot() throws IOException { SnapshotOfRegionAssignmentFromMeta currentAssignmentShapshot = - new SnapshotOfRegionAssignmentFromMeta(new CatalogTracker(this.conf)); + new SnapshotOfRegionAssignmentFromMeta(new CatalogTracker(getConnection())); currentAssignmentShapshot.initialize(); return currentAssignmentShapshot; } @@ -659,7 +668,6 @@ public class RegionPlacementMaintainer { // Get the region to region server map Map> currentAssignment = this.getRegionAssignmentSnapshot().getRegionServerToRegionMap(); - HConnection connection = this.getHBaseAdmin().getConnection(); // track of the failed and succeeded updates int succeededNum = 0; @@ -690,7 +698,7 @@ public class RegionPlacementMaintainer { } if (singleServerPlan != null) { // Update the current region server with its updated favored nodes - BlockingInterface currentRegionServer = connection.getAdmin(entry.getKey()); + BlockingInterface currentRegionServer = getConnection().getAdmin(entry.getKey()); UpdateFavoredNodesRequest request = RequestConverter.buildUpdateFavoredNodesRequest(regionUpdateInfos); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index 72f7dc4..4bba3cb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -190,7 +190,7 @@ public class ServerManager { Configuration c = master.getConfiguration(); maxSkew = c.getLong("hbase.master.maxclockskew", 30000); warningSkew = c.getLong("hbase.master.warningclockskew", 10000); - this.connection = connect ? HConnectionManager.getConnection(c) : null; + this.connection = connect ? HConnectionManager.createConnection(c) : null; } /** @@ -271,7 +271,6 @@ public class ServerManager { * Check is a server of same host and port already exists, * if not, or the existed one got a smaller start code, record it. * - * @param sn the server to check and record * @param sl the server load on the server * @return true if the server is recorded, otherwise, false */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 369d3fe..6745e03 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -87,6 +87,7 @@ import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.ConnectionUtils; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; @@ -733,7 +734,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa blockAndCheckIfStopped(this.clusterStatusTracker); // Create the catalog tracker and start it; - this.catalogTracker = new CatalogTracker(this.zooKeeper, this.conf, this); + // Okay to create and abandon an HConnection instance because CatalogTracker#stop() closes it. + this.catalogTracker = new CatalogTracker(this.zooKeeper, + HConnectionManager.createConnection(conf), this); catalogTracker.start(); // Retrieve clusterId diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java index e01bb00..03efc51 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java @@ -35,12 +35,10 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SplitLogCounters; import org.apache.hadoop.hbase.SplitLogTask; -import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.RetriesExhaustedException; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.executor.ExecutorService; @@ -104,7 +102,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { private boolean workerInGrabTask = false; private final int report_period; private RegionServerServices server = null; - private Configuration conf = null; protected final AtomicInteger tasksInProgress = new AtomicInteger(0); private int maxConcurrentTasks = 0; @@ -116,7 +113,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { this.splitTaskExecutor = splitTaskExecutor; report_period = conf.getInt("hbase.splitlog.report.period", conf.getInt("hbase.splitlog.manager.timeout", SplitLogManager.DEFAULT_TIMEOUT) / 3); - this.conf = conf; this.executorService = this.server.getExecutorService(); this.maxConcurrentTasks = conf.getInt("hbase.regionserver.wal.max.splitters", DEFAULT_MAX_SPLITTERS); @@ -175,11 +171,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { try { LOG.info("SplitLogWorker " + this.serverName + " starting"); this.watcher.registerListener(this); - boolean distributedLogReplay = HLogSplitter.isDistributedLogReplay(conf); - if (distributedLogReplay) { - // initialize a new connection for splitlogworker configuration - HConnectionManager.getConnection(conf); - } // wait for master to create the splitLogZnode int res = -1; @@ -302,7 +293,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { */ private void grabTask(String path) { Stat stat = new Stat(); - long t = -1; byte[] data; synchronized (grabTaskLock) { currentTask = path; @@ -378,11 +368,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { * This method is also used to periodically heartbeat the task progress by transitioning the node * from OWNED to OWNED. *

- * @param isFirstTime - * @param zkw - * @param server - * @param task - * @param taskZKVersion * @return non-negative integer value when task can be owned by current region server otherwise -1 */ protected static int attemptToOwnTask(boolean isFirstTime, ZooKeeperWatcher zkw, @@ -443,8 +428,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { /** * Submit a log split task to executor service - * @param curTask - * @param curTaskZKVersion */ void submitTask(final String curTask, final int curTaskZKVersion, final int reportPeriod) { final MutableInt zkVersion = new MutableInt(curTaskZKVersion); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java index 15986d1..7002dd0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java @@ -1799,7 +1799,7 @@ public class HLogSplitter { synchronized (this.tableNameToHConnectionMap) { hconn = this.tableNameToHConnectionMap.get(tableName); if (hconn == null) { - hconn = HConnectionManager.getConnection(conf); + hconn = HConnectionManager.createConnection(conf); this.tableNameToHConnectionMap.put(tableName, hconn); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java index cea5750..838d7f9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java @@ -25,13 +25,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import org.apache.zookeeper.KeeperException; import java.io.IOException; import java.util.List; @@ -137,8 +135,6 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate implements Abo LOG.info("Stopping " + this.zkw); this.zkw.close(); } - // Not sure why we're deleting a connection that we never acquired or used - HConnectionManager.deleteConnection(this.getConf()); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java index 6f8b7f5..2ad5f3e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationHLogReaderManager.java @@ -46,8 +46,6 @@ public class ReplicationHLogReaderManager { /** * Creates the helper but doesn't open any file * Use setInitialPosition after using the constructor if some content needs to be skipped - * @param fs - * @param conf */ public ReplicationHLogReaderManager(FileSystem fs, Configuration conf) { this.fs = fs; @@ -56,7 +54,6 @@ public class ReplicationHLogReaderManager { /** * Opens the file at the current position - * @param path * @return an HLog reader. * @throws IOException */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 6730960..91a4490 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -84,7 +84,6 @@ public class ReplicationSource extends Thread private HConnection conn; private ReplicationQueues replicationQueues; private ReplicationPeers replicationPeers; - private Configuration conf; private ReplicationQueueInfo replicationQueueInfo; // id of the peer cluster this source replicates to private String peerId; @@ -150,48 +149,47 @@ public class ReplicationSource extends Thread final ReplicationSourceManager manager, final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers, final Stoppable stopper, final String peerClusterZnode, final UUID clusterId) throws IOException { + Configuration c = HBaseConfiguration.create(conf); + decorateConf(c); this.stopper = stopper; - this.conf = HBaseConfiguration.create(conf); - decorateConf(); this.replicationQueueSizeCapacity = - this.conf.getLong("replication.source.size.capacity", 1024*1024*64); + c.getLong("replication.source.size.capacity", 1024*1024*64); this.replicationQueueNbCapacity = - this.conf.getInt("replication.source.nb.capacity", 25000); - this.maxRetriesMultiplier = this.conf.getInt("replication.source.maxretriesmultiplier", 10); - this.socketTimeoutMultiplier = this.conf.getInt("replication.source.socketTimeoutMultiplier", + c.getInt("replication.source.nb.capacity", 25000); + this.maxRetriesMultiplier = c.getInt("replication.source.maxretriesmultiplier", 10); + this.socketTimeoutMultiplier = c.getInt("replication.source.socketTimeoutMultiplier", maxRetriesMultiplier * maxRetriesMultiplier); this.queue = new PriorityBlockingQueue( - this.conf.getInt("hbase.regionserver.maxlogs", 32), + c.getInt("hbase.regionserver.maxlogs", 32), new LogsComparator()); - // TODO: This connection is replication specific or we should make it particular to - // replication and make replication specific settings such as compression or codec to use - // passing Cells. - this.conn = HConnectionManager.getConnection(this.conf); - long bandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0); + // TODO: This connection is replication specific. Consider replication + // specific settings such as compression or codec to use passing Cells. + this.conn = HConnectionManager.createConnection(c); + long bandwidth = c.getLong("replication.source.per.peer.node.bandwidth", 0); this.throttler = new ReplicationThrottler((double)bandwidth/10.0); this.replicationQueues = replicationQueues; this.replicationPeers = replicationPeers; this.manager = manager; this.sleepForRetries = - this.conf.getLong("replication.source.sleepforretries", 1000); + c.getLong("replication.source.sleepforretries", 1000); this.fs = fs; this.metrics = new MetricsSource(peerClusterZnode); - this.repLogReader = new ReplicationHLogReaderManager(this.fs, this.conf); + this.repLogReader = new ReplicationHLogReaderManager(this.fs, c); this.clusterId = clusterId; this.peerClusterZnode = peerClusterZnode; this.replicationQueueInfo = new ReplicationQueueInfo(peerClusterZnode); // ReplicationQueueInfo parses the peerId out of the znode for us this.peerId = this.replicationQueueInfo.getPeerId(); - this.replicationSinkMgr = new ReplicationSinkManager(conn, peerId, replicationPeers, this.conf); - this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2); + this.replicationSinkMgr = new ReplicationSinkManager(conn, peerId, replicationPeers, c); + this.logQueueWarnThreshold = c.getInt("replication.source.log.queue.warn", 2); } - private void decorateConf() { - String replicationCodec = this.conf.get(HConstants.REPLICATION_CODEC_CONF_KEY); + private void decorateConf(Configuration conf) { + String replicationCodec = conf.get(HConstants.REPLICATION_CODEC_CONF_KEY); if (StringUtils.isNotEmpty(replicationCodec)) { - this.conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec); + conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java index 8549982..0051f7a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java @@ -77,7 +77,7 @@ public class RegionsResource extends ResourceBase { TableName tableName = TableName.valueOf(tableResource.getName()); TableInfoModel model = new TableInfoModel(tableName.getNameAsString()); Map regions = MetaScanner.allTableRegions( - servlet.getConfiguration(), null, tableName, false); + servlet.getAdmin().getConnection(), tableName, false); for (Map.Entry e: regions.entrySet()) { HRegionInfo hri = e.getKey(); ServerName addr = e.getValue(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index aa63660..0344372 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -303,9 +303,9 @@ public class HBaseFsck extends Configured { */ public void connect() throws IOException { admin = new HBaseAdmin(getConf()); - meta = new HTable(getConf(), TableName.META_TABLE_NAME); - status = admin.getClusterStatus(); connection = admin.getConnection(); + meta = (HTable) connection.getTable(TableName.META_TABLE_NAME); + status = admin.getClusterStatus(); } /** @@ -528,7 +528,7 @@ public class HBaseFsck extends Configured { public void checkRegionBoundaries() { try { ByteArrayComparator comparator = new ByteArrayComparator(); - List regions = MetaScanner.listAllRegions(getConf(), false); + List regions = MetaScanner.listAllRegions(connection, false); final RegionBoundariesInformation currentRegionBoundariesInformation = new RegionBoundariesInformation(); for (HRegionInfo regionInfo : regions) { @@ -2883,7 +2883,7 @@ public class HBaseFsck extends Configured { }; if (!checkMetaOnly) { // Scan hbase:meta to pick up user regions - MetaScanner.metaScan(getConf(), visitor); + MetaScanner.metaScan(connection, visitor); } errors.print(""); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java index 7ed1530..739e97b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java @@ -54,6 +54,8 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.catalog.MetaReader; import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.NoServerForRegionException; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; @@ -144,7 +146,7 @@ public class RegionSplitter { * {@link HexStringSplit} to partition their table and set it as default, but * provided this for your custom algorithm. To use, create a new derived class * from this interface and call {@link RegionSplitter#createPresplitTable} or - * {@link RegionSplitter#rollingSplit(String, SplitAlgorithm, Configuration)} with the + * {@link RegionSplitter#rollingSplit(HConnection, String, SplitAlgorithm)} with the * argument splitClassName giving the name of your class. */ public interface SplitAlgorithm { @@ -283,6 +285,7 @@ public class RegionSplitter { public static void main(String[] args) throws IOException, InterruptedException, ParseException { Configuration conf = HBaseConfiguration.create(); + HConnection conn = null; // parse user input Options opt = new Options(); @@ -351,22 +354,32 @@ public class RegionSplitter { } if (createTable) { - conf.set("split.count", cmd.getOptionValue("c")); - createPresplitTable(tableName, splitAlgo, cmd.getOptionValue("f").split(":"), conf); + try { + conf.set("split.count", cmd.getOptionValue("c")); + conn = HConnectionManager.createConnection(conf); + createPresplitTable(conn, tableName, splitAlgo, cmd.getOptionValue("f").split(":")); + } finally { + if (conn != null) conn.close(); + } } if (rollingSplit) { if (cmd.hasOption("o")) { conf.set("split.outstanding", cmd.getOptionValue("o")); } - rollingSplit(tableName, splitAlgo, conf); + try { + conn = HConnectionManager.createConnection(conf); + rollingSplit(conn, tableName, splitAlgo); + } finally { + if (conn != null) conn.close(); + } } } - static void createPresplitTable(String tableName, SplitAlgorithm splitAlgo, - String[] columnFamilies, Configuration conf) throws IOException, + static void createPresplitTable(HConnection conn, String tableName, SplitAlgorithm splitAlgo, + String[] columnFamilies) throws IOException, InterruptedException { - final int splitCount = conf.getInt("split.count", 0); + final int splitCount = conn.getConfiguration().getInt("split.count", 0); Preconditions.checkArgument(splitCount > 1, "Split count must be > 1"); Preconditions.checkArgument(columnFamilies.length > 0, @@ -378,17 +391,17 @@ public class RegionSplitter { for (String cf : columnFamilies) { desc.addFamily(new HColumnDescriptor(Bytes.toBytes(cf))); } - HBaseAdmin admin = new HBaseAdmin(conf); + HBaseAdmin admin = new HBaseAdmin(conn); Preconditions.checkArgument(!admin.tableExists(tableName), "Table already exists: " + tableName); admin.createTable(desc, splitAlgo.split(splitCount)); admin.close(); LOG.debug("Table created! Waiting for regions to show online in META..."); - if (!conf.getBoolean("split.verify", true)) { + if (!conn.getConfiguration().getBoolean("split.verify", true)) { // NOTE: createTable is synchronous on the table, but not on the regions int onlineRegions = 0; while (onlineRegions < splitCount) { - onlineRegions = MetaReader.getRegionCount(conf, tableName); + onlineRegions = MetaReader.getRegionCount(conn, tableName); LOG.debug(onlineRegions + " of " + splitCount + " regions online..."); if (onlineRegions < splitCount) { Thread.sleep(10 * 1000); // sleep @@ -399,20 +412,20 @@ public class RegionSplitter { LOG.debug("Finished creating table with " + splitCount + " regions"); } - static void rollingSplit(String tableName, SplitAlgorithm splitAlgo, - Configuration conf) throws IOException, InterruptedException { - final int minOS = conf.getInt("split.outstanding", 2); + static void rollingSplit(HConnection conn, String tableName, SplitAlgorithm splitAlgo) + throws IOException, InterruptedException { + final int minOS = conn.getConfiguration().getInt("split.outstanding", 2); - HTable table = new HTable(conf, tableName); + HTable table = (HTable) conn.getTable(tableName); // max outstanding splits. default == 50% of servers final int MAX_OUTSTANDING = Math.max(table.getConnection().getCurrentNrHRS() / 2, minOS); - Path hbDir = FSUtils.getRootDir(conf); + Path hbDir = FSUtils.getRootDir(conn.getConfiguration()); Path tableDir = FSUtils.getTableDir(hbDir, table.getName()); Path splitFile = new Path(tableDir, "_balancedSplit"); - FileSystem fs = FileSystem.get(conf); + FileSystem fs = FileSystem.get(conn.getConfiguration()); // get a list of daughter regions to create LinkedList> tmpRegionSet = getSplits(table, splitAlgo); @@ -532,7 +545,7 @@ public class RegionSplitter { admin.split(table.getTableName(), split); LinkedList> finished = Lists.newLinkedList(); - if (conf.getBoolean("split.verify", true)) { + if (conn.getConfiguration().getBoolean("split.verify", true)) { // we need to verify and rate-limit our splits outstanding.addLast(dr); // with too many outstanding splits, wait for some to finish @@ -563,7 +576,7 @@ public class RegionSplitter { } } } - if (conf.getBoolean("split.verify", true)) { + if (conn.getConfiguration().getBoolean("split.verify", true)) { while (!outstanding.isEmpty()) { LinkedList> finished = splitScan(outstanding, table, splitAlgo); diff --git a/hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp b/hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp index be9c750..d2449fc 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp @@ -21,7 +21,6 @@ import="java.util.Date" import="org.apache.hadoop.conf.Configuration" import="org.apache.hadoop.hbase.client.HBaseAdmin" - import="org.apache.hadoop.hbase.client.HConnectionManager" import="org.apache.hadoop.hbase.master.HMaster" import="org.apache.hadoop.hbase.snapshot.SnapshotInfo" import="org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription" @@ -189,8 +188,6 @@ <% } %> <% } // end else - -HConnectionManager.deleteConnection(hbadmin.getConfiguration()); %> diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp index 8df53cb..230e256 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -24,7 +24,6 @@ import="org.apache.hadoop.conf.Configuration" import="org.apache.hadoop.hbase.client.HTable" import="org.apache.hadoop.hbase.client.HBaseAdmin" - import="org.apache.hadoop.hbase.client.HConnectionManager" import="org.apache.hadoop.hbase.HRegionInfo" import="org.apache.hadoop.hbase.ServerName" import="org.apache.hadoop.hbase.ServerLoad" @@ -314,8 +313,6 @@ ex.printStackTrace(System.err); } } // end else - -HConnectionManager.deleteConnection(hbadmin.getConfiguration()); %> diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java index 1f6d666..45414ff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java @@ -95,7 +95,7 @@ public abstract class HBaseCluster implements Closeable, Configurable { } /** - * Returns an {@link MasterAdminService.BlockingInterface} to the active master + * Returns an {@link MasterService.BlockingInterface} to the active master */ public abstract MasterService.BlockingInterface getMaster() throws IOException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index be16e30..0833c1e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -59,6 +59,8 @@ import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionImplForTests; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -223,7 +225,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { bloomAndCompressionCombinations(); public HBaseTestingUtility() { - this(HBaseConfiguration.create()); + super(); } public HBaseTestingUtility(Configuration conf) { @@ -259,22 +261,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return htu; } - /** - * Returns this classes's instance of {@link Configuration}. Be careful how - * you use the returned Configuration since {@link HConnection} instances - * can be shared. The Map of HConnections is keyed by the Configuration. If - * say, a Connection was being used against a cluster that had been shutdown, - * see {@link #shutdownMiniCluster()}, then the Connection will no longer - * be wholesome. Rather than use the return direct, its usually best to - * make a copy and use that. Do - * Configuration c = new Configuration(INSTANCE.getConfiguration()); - * @return Instance of Configuration. - */ - @Override - public Configuration getConfiguration() { - return super.getConfiguration(); - } - public void setHBaseCluster(HBaseCluster hbaseCluster) { this.hbaseCluster = hbaseCluster; } @@ -962,6 +948,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { hbaseAdmin = null; } + if (conn != null) { + ((HConnectionImplForTests) conn).close0(); + conn = null; + } + // unset the configuration for MIN and MAX RS to start conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1); conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, -1); @@ -1040,10 +1031,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Create a table. - * @param tableName - * @param family * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(String tableName, String family) throws IOException{ @@ -1052,10 +1040,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Create a table. - * @param tableName - * @param family * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(byte[] tableName, byte[] family) throws IOException{ @@ -1064,10 +1049,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Create a table. - * @param tableName - * @param families * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(TableName tableName, String[] families) throws IOException { @@ -1080,10 +1062,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Create a table. - * @param tableName - * @param family * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(TableName tableName, byte[] family) throws IOException{ @@ -1093,42 +1072,73 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Create a table. - * @param tableName - * @param families * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(byte[] tableName, byte[][] families) throws IOException { - return createTable(tableName, families, - new Configuration(getConfiguration())); + return createTable(TableName.valueOf(tableName), families); } /** - * Create a table. - * @param tableName - * @param families + * Create a table using the implicit connection instance. * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(TableName tableName, byte[][] families) throws IOException { - return createTable(tableName, families, - new Configuration(getConfiguration())); + HTableDescriptor desc = new HTableDescriptor(tableName); + for(byte[] family : families) { + HColumnDescriptor hcd = new HColumnDescriptor(family); + // Disable blooms (they are on by default as of 0.95) but we disable them here because + // tests have hard coded counts of what to expect in block cache, etc., and blooms being + // on is interfering. + hcd.setBloomFilterType(BloomType.NONE); + desc.addFamily(hcd); + } + getHBaseAdmin().createTable(desc); + return (HTable) getConnection().getTable(tableName); } + /** + * Create a table using the implicit connection instance. + * @return An HTable instance for the created table. + */ + public HTable createTable(TableName tableName, byte[][] families, byte[][] splitRows) + throws IOException { + HTableDescriptor desc = new HTableDescriptor(tableName); + for(byte[] family:families) { + HColumnDescriptor hcd = new HColumnDescriptor(family); + desc.addFamily(hcd); + } + getHBaseAdmin().createTable(desc, splitRows); + // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned + waitUntilAllRegionsAssigned(tableName); + return (HTable) getConnection().getTable(tableName); + } + + /** + * Create a table using the implicit connection instance. + * @return An HTable instance for the created table. + */ public HTable createTable(byte[] tableName, byte[][] families, int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException { return createTable(TableName.valueOf(tableName), families, numVersions, startKey, endKey, numRegions); } + /** + * Create a table using the implicit connection instance. + * @return An HTable instance for the created table. + */ public HTable createTable(String tableName, byte[][] families, int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException { return createTable(TableName.valueOf(tableName), families, numVersions, startKey, endKey, numRegions); } + /** + * Create a table using the implicit connection instance. + * @return An HTable instance for the created table. + */ public HTable createTable(TableName tableName, byte[][] families, int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException{ @@ -1141,19 +1151,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc, startKey, endKey, numRegions); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(getConfiguration(), tableName); + return (HTable) getConnection().getTable(tableName); } /** * Create a table. - * @param htd - * @param families - * @param c Configuration to use * @return An HTable instance for the created table. - * @throws IOException */ - public HTable createTable(HTableDescriptor htd, byte[][] families, Configuration c) - throws IOException { + public HTable createTable(HTableDescriptor htd, byte[][] families) + throws IOException { for(byte[] family : families) { HColumnDescriptor hcd = new HColumnDescriptor(family); // Disable blooms (they are on by default as of 0.95) but we disable them here because @@ -1165,140 +1171,46 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(htd); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(htd.getTableName()); - return new HTable(c, htd.getTableName()); - } - - /** - * Create a table. - * @param tableName - * @param families - * @param c Configuration to use - * @return An HTable instance for the created table. - * @throws IOException - */ - public HTable createTable(TableName tableName, byte[][] families, - final Configuration c) - throws IOException { - return createTable(new HTableDescriptor(tableName), families, c); + return (HTable) getConnection().getTable(htd.getTableName()); } /** * Create a table. - * @param tableName - * @param families - * @param c Configuration to use * @return An HTable instance for the created table. - * @throws IOException */ - public HTable createTable(byte[] tableName, byte[][] families, - final Configuration c) + public HTable createTable(byte[] tableName, byte[] family, int numVersions) throws IOException { - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); - for(byte[] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family); - // Disable blooms (they are on by default as of 0.95) but we disable them here because - // tests have hard coded counts of what to expect in block cache, etc., and blooms being - // on is interfering. - hcd.setBloomFilterType(BloomType.NONE); - desc.addFamily(hcd); - } - getHBaseAdmin().createTable(desc); - return new HTable(c, tableName); + return createTable(TableName.valueOf(tableName), new byte[][]{family}, numVersions); } /** * Create a table. - * @param tableName - * @param families - * @param c Configuration to use - * @param numVersions * @return An HTable instance for the created table. - * @throws IOException */ - public HTable createTable(TableName tableName, byte[][] families, - final Configuration c, int numVersions) + public HTable createTable(TableName tableName, byte[] family, int numVersions) throws IOException { - HTableDescriptor desc = new HTableDescriptor(tableName); - for(byte[] family : families) { - HColumnDescriptor hcd = new HColumnDescriptor(family) - .setMaxVersions(numVersions); - desc.addFamily(hcd); - } - getHBaseAdmin().createTable(desc); - // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned - waitUntilAllRegionsAssigned(tableName); - return new HTable(c, tableName); + return createTable(tableName, new byte[][]{family}, numVersions); } /** * Create a table. - * @param tableName - * @param families - * @param c Configuration to use - * @param numVersions * @return An HTable instance for the created table. - * @throws IOException */ - public HTable createTable(byte[] tableName, byte[][] families, - final Configuration c, int numVersions) + public HTable createTable(byte[] tableName, byte[][] families, int numVersions) throws IOException { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); for(byte[] family : families) { HColumnDescriptor hcd = new HColumnDescriptor(family) - .setMaxVersions(numVersions); + .setMaxVersions(numVersions); desc.addFamily(hcd); } getHBaseAdmin().createTable(desc); - return new HTable(c, tableName); - } - - /** - * Create a table. - * @param tableName - * @param family - * @param numVersions - * @return An HTable instance for the created table. - * @throws IOException - */ - public HTable createTable(byte[] tableName, byte[] family, int numVersions) - throws IOException { - return createTable(tableName, new byte[][]{family}, numVersions); - } - - /** - * Create a table. - * @param tableName - * @param family - * @param numVersions - * @return An HTable instance for the created table. - * @throws IOException - */ - public HTable createTable(TableName tableName, byte[] family, int numVersions) - throws IOException { - return createTable(tableName, new byte[][]{family}, numVersions); - } - - /** - * Create a table. - * @param tableName - * @param families - * @param numVersions - * @return An HTable instance for the created table. - * @throws IOException - */ - public HTable createTable(byte[] tableName, byte[][] families, - int numVersions) - throws IOException { - return createTable(TableName.valueOf(tableName), families, numVersions); + return (HTable) getConnection().getTable(tableName); } /** * Create a table. - * @param tableName - * @param families - * @param numVersions * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(TableName tableName, byte[][] families, int numVersions) @@ -1311,16 +1223,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(new Configuration(getConfiguration()), tableName); + return (HTable) getConnection().getTable(tableName); } /** * Create a table. - * @param tableName - * @param families - * @param numVersions * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(byte[] tableName, byte[][] families, int numVersions, int blockSize) throws IOException { @@ -1330,11 +1238,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Create a table. - * @param tableName - * @param families - * @param numVersions * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(TableName tableName, byte[][] families, int numVersions, int blockSize) throws IOException { @@ -1348,16 +1252,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(new Configuration(getConfiguration()), tableName); + return (HTable) getConnection().getTable(tableName); } /** * Create a table. - * @param tableName - * @param families - * @param numVersions * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(byte[] tableName, byte[][] families, int[] numVersions) @@ -1367,11 +1267,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Create a table. - * @param tableName - * @param families - * @param numVersions * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(TableName tableName, byte[][] families, int[] numVersions) @@ -1387,16 +1283,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(new Configuration(getConfiguration()), tableName); + return (HTable) getConnection().getTable(tableName); } /** * Create a table. - * @param tableName - * @param family - * @param splitRows * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(byte[] tableName, byte[] family, byte[][] splitRows) throws IOException{ @@ -1405,11 +1297,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Create a table. - * @param tableName - * @param family - * @param splitRows * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(TableName tableName, byte[] family, byte[][] splitRows) throws IOException { @@ -1419,28 +1307,16 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc, splitRows); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(getConfiguration(), tableName); + return (HTable) getConnection().getTable(tableName); } /** * Create a table. - * @param tableName - * @param families - * @param splitRows * @return An HTable instance for the created table. - * @throws IOException */ public HTable createTable(byte[] tableName, byte[][] families, byte[][] splitRows) throws IOException { - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); - for(byte[] family:families) { - HColumnDescriptor hcd = new HColumnDescriptor(family); - desc.addFamily(hcd); - } - getHBaseAdmin().createTable(desc, splitRows); - // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned - waitUntilAllRegionsAssigned(TableName.valueOf(tableName)); - return new HTable(getConfiguration(), tableName); + return createTable(TableName.valueOf(tableName), families, splitRows); } /** @@ -1488,7 +1364,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { public static final String START_KEY = new String(START_KEY_BYTES, HConstants.UTF8_CHARSET); /** - * Create a table of name name with {@link COLUMNS} for + * Create a table of name name with {@link #COLUMNS} for * families. * @param name Name to give table. * @param versions How many versions to allow per column. @@ -1510,7 +1386,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } /** - * Create a table of name name with {@link COLUMNS} for + * Create a table of name name with {@link #COLUMNS} for * families. * @param name Name to give table. * @return Column descriptor. @@ -2371,6 +2247,40 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return hbaseCluster; } + /** This is intended to be an instance of {@link HConnectionImplForTests} */ + private HConnection conn = null; + + /** + * Retrieve a connection to the cluster. This is an instance of + * {@link HConnectionImplForTests}, not a standard connection implementation. + * The reason being, for testing purposes, it's preferable to allow the test + * harness to manage the connection instance. + *

+ * Note this means that {@code getConfiguration() != getConnection().getConfiguration()} + *

+ * Tests that verify resource cleanup should instead use their own connection + * instance: + * {@code + * Configuration conf = TEST_UTIL.getConfiguration(); + * HConnection conn = HConnectionManager.createConnection(conf); + * ... + * conn.close(); + * } + * + * @return + * @throws IOException + */ + public synchronized HConnection getConnection() throws IOException { + if (conn == null) { + // inject our customed HConnection, one that doesn't close when requested. + Configuration conf = HBaseConfiguration.create(getConfiguration()); + conf.set("hbase.client.connection.impl", + "org.apache.hadoop.hbase.client.HConnectionImplForTests"); + conn = HConnectionManager.createConnection(conf); + } + return conn; + } + /** * Returns a HBaseAdmin instance. * This instance is shared between HBaseTestingUtility instance users. @@ -2383,15 +2293,14 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { public synchronized HBaseAdmin getHBaseAdmin() throws IOException { if (hbaseAdmin == null){ - hbaseAdmin = new HBaseAdminForTests(getConfiguration()); + hbaseAdmin = new HBaseAdminForTests(getConnection()); } return hbaseAdmin; } private HBaseAdminForTests hbaseAdmin = null; private static class HBaseAdminForTests extends HBaseAdmin { - public HBaseAdminForTests(Configuration c) throws MasterNotRunningException, - ZooKeeperConnectionException, IOException { + public HBaseAdminForTests(HConnection c) throws IOException { super(c); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java index 834fc51..968ce49 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java @@ -29,7 +29,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; @@ -494,7 +493,6 @@ public class MiniHBaseCluster extends HBaseCluster { if (this.hbaseCluster != null) { this.hbaseCluster.shutdown(); } - HConnectionManager.deleteAllConnections(false); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ServerResourceCheckerJUnitListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ServerResourceCheckerJUnitListener.java deleted file mode 100644 index 4e01b5e..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ServerResourceCheckerJUnitListener.java +++ /dev/null @@ -1,42 +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; - -import org.apache.hadoop.hbase.ResourceChecker.Phase; -import org.apache.hadoop.hbase.client.HConnectionTestingUtility; - -/** - * Monitor the resources. use by the tests All resources in {@link ResourceCheckerJUnitListener} - * plus the number of connection. - */ -public class ServerResourceCheckerJUnitListener extends ResourceCheckerJUnitListener { - - static class ConnectionCountResourceAnalyzer extends ResourceChecker.ResourceAnalyzer { - @Override - public int getVal(Phase phase) { - return HConnectionTestingUtility.getConnectionCount(); - } - } - - @Override - protected void addResourceAnalyzer(ResourceChecker rc) { - rc.addResourceAnalyzer(new ConnectionCountResourceAnalyzer()); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java index 0294b12..bf096f2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java @@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.catalog.MetaReader; import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -94,11 +95,11 @@ public class TestRegionRebalancing { @Test (timeout=300000) public void testRebalanceOnRegionServerNumberChange() throws IOException, InterruptedException { - HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration()); + HBaseAdmin admin = UTIL.getHBaseAdmin(); admin.createTable(this.desc, Arrays.copyOfRange(HBaseTestingUtility.KEYS, 1, HBaseTestingUtility.KEYS.length)); - this.table = new HTable(UTIL.getConfiguration(), this.desc.getTableName()); - CatalogTracker ct = new CatalogTracker(UTIL.getConfiguration()); + this.table = (HTable) UTIL.getConnection().getTable(desc.getTableName()); + CatalogTracker ct = new CatalogTracker(UTIL.getConnection()); ct.start(); try { MetaReader.fullScanMetaAndPrint(ct); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java index 3a31c15..5288439 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java @@ -69,8 +69,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; - - @Category(LargeTests.class) public class TestZooKeeper { private final Log LOG = LogFactory.getLog(this.getClass()); @@ -140,7 +138,7 @@ public class TestZooKeeper { // We don't want to share the connection as we will check its state c.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "1111"); - HConnection connection = HConnectionManager.getConnection(c); + HConnection connection = HConnectionManager.createConnection(c); ZooKeeperWatcher connectionZK = getZooKeeperWatcher(connection); LOG.info("ZooKeeperWatcher= 0x"+ Integer.toHexString( @@ -280,13 +278,15 @@ public class TestZooKeeper { ipMeta.exists(new Get(row)); // make sure they aren't the same - ZooKeeperWatcher z1 = - getZooKeeperWatcher(HConnectionManager.getConnection(localMeta.getConfiguration())); - ZooKeeperWatcher z2 = - getZooKeeperWatcher(HConnectionManager.getConnection(otherConf)); + HConnection c1 = HConnectionManager.createConnection(localMeta.getConfiguration()); + ZooKeeperWatcher z1 = getZooKeeperWatcher(c1); + HConnection c2 = HConnectionManager.createConnection(otherConf); + ZooKeeperWatcher z2 = getZooKeeperWatcher(c2); assertFalse(z1 == z2); assertFalse(z1.getQuorum().equals(z2.getQuorum())); + c1.close(); + c2.close(); localMeta.close(); ipMeta.close(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/MetaMockingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/MetaMockingUtil.java index 9177073..d7ee9a5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/MetaMockingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/MetaMockingUtil.java @@ -40,19 +40,7 @@ public class MetaMockingUtil { * Returns a Result object constructed from the given region information simulating * a catalog table result. * @param region the HRegionInfo object or null - * @return A mocked up Result that fakes a Get on a row in the hbase:meta table. - * @throws IOException - */ - public static Result getMetaTableRowResult(final HRegionInfo region) - throws IOException { - return getMetaTableRowResult(region, null, null, null); - } - - /** - * Returns a Result object constructed from the given region information simulating - * a catalog table result. - * @param region the HRegionInfo object or null - * @param ServerName to use making startcode and server hostname:port in meta or null + * @param sn to use making startcode and server hostname:port in meta or null * @return A mocked up Result that fakes a Get on a row in the hbase:meta table. * @throws IOException */ @@ -65,7 +53,7 @@ public class MetaMockingUtil { * Returns a Result object constructed from the given region information simulating * a catalog table result. * @param region the HRegionInfo object or null - * @param ServerName to use making startcode and server hostname:port in meta or null + * @param sn to use making startcode and server hostname:port in meta or null * @param splita daughter region or null * @param splitb daughter region or null * @return A mocked up Result that fakes a Get on a row in the hbase:meta table. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java index caeafbd..256bc85 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; @@ -111,16 +110,12 @@ public class TestCatalogTracker { LOG.warn("Unable to delete hbase:meta location", e); } - // Clear out our doctored connection or could mess up subsequent tests. - HConnectionManager.deleteConnection(UTIL.getConfiguration()); - this.watcher.close(); } private CatalogTracker constructAndStartCatalogTracker(final HConnection c) throws IOException, InterruptedException { - CatalogTracker ct = new CatalogTracker(this.watcher, UTIL.getConfiguration(), - c, this.abortable); + CatalogTracker ct = new CatalogTracker(this.watcher, c, this.abortable); ct.start(); return ct; } @@ -303,8 +298,7 @@ public class TestCatalogTracker { * {@link HConnection#getAdmin(ServerName)} is called, returns the passed * {@link ClientProtos.ClientService.BlockingInterface} instance when * {@link HConnection#getClient(ServerName)} is called (Be sure to call - * {@link HConnectionManager#deleteConnection(org.apache.hadoop.conf.Configuration)} - * when done with this mocked Connection. + * {@link HConnection#close()} when done with this mocked Connection. * @throws IOException */ private HConnection mockConnection(final AdminProtos.AdminService.BlockingInterface admin, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java index 6ed0445..5fe49f4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditor.java @@ -54,7 +54,6 @@ import org.junit.experimental.categories.Category; public class TestMetaReaderEditor { private static final Log LOG = LogFactory.getLog(TestMetaReaderEditor.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); - private static ZooKeeperWatcher zkw; private static CatalogTracker CT; private final static Abortable ABORTABLE = new Abortable() { private final AtomicBoolean abort = new AtomicBoolean(false); @@ -79,8 +78,8 @@ public class TestMetaReaderEditor { // responsive. 1 second is default as is ten retries. c.setLong("hbase.client.pause", 1000); c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 10); - zkw = new ZooKeeperWatcher(c, "TestMetaReaderEditor", ABORTABLE); - CT = new CatalogTracker(zkw, c, ABORTABLE); + ZooKeeperWatcher zkw = new ZooKeeperWatcher(c, "TestMetaReaderEditor", ABORTABLE); + CT = new CatalogTracker(zkw, UTIL.getConnection(), ABORTABLE); CT.start(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java index 6ae0ecd..acaee85 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; -import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; @@ -133,13 +132,14 @@ public class TestMetaReaderEditorNoCluster { @Test public void testRideOverServerNotRunning() throws IOException, InterruptedException, ServiceException { + // TODO: fix me! // Need a zk watcher. ZooKeeperWatcher zkw = new ZooKeeperWatcher(UTIL.getConfiguration(), this.getClass().getSimpleName(), ABORTABLE, true); // This is a servername we use in a few places below. ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis()); - HConnection connection; + HConnection connection = null; CatalogTracker ct = null; try { // Mock an ClientProtocol. Our mock implementation will fail a few @@ -185,7 +185,7 @@ public class TestMetaReaderEditorNoCluster { // Associate a spied-upon HConnection with UTIL.getConfiguration. Need // to shove this in here first so it gets picked up all over; e.g. by // HTable. - connection = HConnectionTestingUtility.getSpiedConnection(UTIL.getConfiguration()); + connection = Mockito.spy(HConnectionManager.createConnection(UTIL.getConfiguration())); // Fix the location lookup so it 'works' though no network. First // make an 'any location' object. final HRegionLocation anyLocation = @@ -205,7 +205,7 @@ public class TestMetaReaderEditorNoCluster { when(connection).getClient(Mockito.any(ServerName.class)); // Now start up the catalogtracker with our doctored Connection. - ct = new CatalogTracker(zkw, null, connection, ABORTABLE); + ct = new CatalogTracker(zkw, connection, ABORTABLE); ct.start(); // Scan meta for user tables and verify we got back expected answer. NavigableMap hris = MetaReader.getServerUserRegions(ct, sn); @@ -218,7 +218,7 @@ public class TestMetaReaderEditorNoCluster { scan((RpcController)Mockito.any(), (ScanRequest)Mockito.any()); } finally { if (ct != null) ct.stop(); - HConnectionManager.deleteConnection(UTIL.getConfiguration()); + if (connection != null) connection.close(); zkw.close(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionImplForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionImplForTests.java new file mode 100644 index 0000000..bd3ccd2 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionImplForTests.java @@ -0,0 +1,47 @@ +/** + * + * 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 org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.security.User; + +import java.io.IOException; +import java.util.concurrent.ExecutorService; + +/** + * This implementation defers closure until the harness is shut down. Modeled + * after {@link HBaseTestingUtility.HBaseAdminForTests} + */ +public class HConnectionImplForTests extends ConnectionManager.HConnectionImplementation { + HConnectionImplForTests(Configuration conf, ExecutorService pool, User user) + throws IOException { + super(conf, pool, user); + } + + @Override + public void close() { + LOG.warn("close() called on HConnection instance returned from HBaseTestingUtility.getConnection()"); + } + + /** Actually closes the connection */ + public void close0() { + super.close(); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java index 52fb407..1ac4286 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java @@ -43,34 +43,20 @@ public class HConnectionTestingUtility { * Get a Mocked {@link HConnection} that goes with the passed conf * configuration instance. Minimally the mock will return * conf when {@link HConnection#getConfiguration()} is invoked. - * Be sure to shutdown the connection when done by calling - * {@link HConnectionManager#deleteConnection(Configuration)} else it - * will stick around; this is probably not what you want. * @param conf configuration * @return HConnection object for conf - * @throws ZooKeeperConnectionException */ - public static ClusterConnection getMockedConnection(final Configuration conf) - throws ZooKeeperConnectionException { - HConnectionKey connectionKey = new HConnectionKey(conf); - synchronized (ConnectionManager.CONNECTION_INSTANCES) { - HConnectionImplementation connection = - ConnectionManager.CONNECTION_INSTANCES.get(connectionKey); - if (connection == null) { - connection = Mockito.mock(HConnectionImplementation.class); - Mockito.when(connection.getConfiguration()).thenReturn(conf); - ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection); - } - return connection; - } + public static ClusterConnection getMockedConnection(final Configuration conf) { + ClusterConnection conn = Mockito.mock(HConnectionImplementation.class); + Mockito.when(conn.getConfiguration()).thenReturn(conf); + return conn; } /** * Calls {@link #getMockedConnection(Configuration)} and then mocks a few * more of the popular {@link HConnection} methods so they do 'normal' * operation (see return doc below for list). Be sure to shutdown the - * connection when done by calling - * {@link HConnectionManager#deleteConnection(Configuration)} else it + * connection when done by calling {@link HConnection#close()} else it * will stick around; this is probably not what you want. * * @param conf Configuration to use @@ -88,9 +74,7 @@ public class HConnectionTestingUtility { * and that returns the passed {@link AdminProtos.AdminService.BlockingInterface} instance when * {@link HConnection#getAdmin(ServerName)} is called, returns the passed * {@link ClientProtos.ClientService.BlockingInterface} instance when - * {@link HConnection#getClient(ServerName)} is called (Be sure to call - * {@link HConnectionManager#deleteConnection(Configuration)} - * when done with this mocked Connection. + * {@link HConnection#getClient(ServerName)} is called. * @throws IOException */ public static ClusterConnection getMockedConnectionAndDecorate(final Configuration conf, @@ -98,7 +82,7 @@ public class HConnectionTestingUtility { final ClientProtos.ClientService.BlockingInterface client, final ServerName sn, final HRegionInfo hri) throws IOException { - ClusterConnection c = HConnectionTestingUtility.getMockedConnection(conf); + ClusterConnection c = getMockedConnection(conf); Mockito.doNothing().when(c).close(); // Make it so we return a particular location when asked. final HRegionLocation loc = new HRegionLocation(hri, sn); @@ -123,39 +107,4 @@ public class HConnectionTestingUtility { c, conf, null, RpcRetryingCallerFactory.instantiate(conf), false)); return c; } - - /** - * Get a Mockito spied-upon {@link HConnection} that goes with the passed - * conf configuration instance. - * Be sure to shutdown the connection when done by calling - * {@link HConnectionManager#deleteConnection(Configuration)} else it - * will stick around; this is probably not what you want. - * @param conf configuration - * @return HConnection object for conf - * @throws ZooKeeperConnectionException - * @see @link - * {http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html#spy(T)} - */ - public static HConnection getSpiedConnection(final Configuration conf) - throws IOException { - HConnectionKey connectionKey = new HConnectionKey(conf); - synchronized (ConnectionManager.CONNECTION_INSTANCES) { - HConnectionImplementation connection = - ConnectionManager.CONNECTION_INSTANCES.get(connectionKey); - if (connection == null) { - connection = Mockito.spy(new HConnectionImplementation(conf, true)); - ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection); - } - return connection; - } - } - - /** - * @return Count of extant connection instances - */ - public static int getConnectionCount() { - synchronized (ConnectionManager.CONNECTION_INSTANCES) { - return ConnectionManager.CONNECTION_INSTANCES.size(); - } - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index ea56574..68bdee1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -26,8 +26,6 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -58,7 +56,6 @@ import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.catalog.CatalogTracker; -import org.apache.hadoop.hbase.catalog.MetaReader; import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.master.AssignmentManager; @@ -864,7 +861,7 @@ public class TestAdmin { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc, splitKeys); - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName); Map regions = ht.getRegionLocations(); ht.close(); assertEquals("Tried to create " + expectedRegions + " regions " @@ -1619,20 +1616,6 @@ public class TestAdmin { } /** - * HBASE-4417 checkHBaseAvailable() doesn't close zk connections - */ - @Test (timeout=300000) - public void testCheckHBaseAvailableClosesConnection() throws Exception { - Configuration conf = TEST_UTIL.getConfiguration(); - - int initialCount = HConnectionTestingUtility.getConnectionCount(); - HBaseAdmin.checkHBaseAvailable(conf); - int finalCount = HConnectionTestingUtility.getConnectionCount(); - - Assert.assertEquals(initialCount, finalCount) ; - } - - /** * Check that we have an exception if the cluster is not there. */ @Test (timeout=300000) @@ -1643,8 +1626,6 @@ public class TestAdmin { conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, 9999)+10); - int initialCount = HConnectionTestingUtility.getConnectionCount(); - long start = System.currentTimeMillis(); try { HBaseAdmin.checkHBaseAvailable(conf); @@ -1656,10 +1637,6 @@ public class TestAdmin { } long end = System.currentTimeMillis(); - int finalCount = HConnectionTestingUtility.getConnectionCount(); - - Assert.assertEquals(initialCount, finalCount) ; - LOG.info("It took "+(end-start)+" ms to find out that" + " HBase was not available"); } @@ -1701,7 +1678,7 @@ public class TestAdmin { final byte [] nameBytes = Bytes.toBytes(name); HTable t = TEST_UTIL.createTable(nameBytes, HConstants.CATALOG_FAMILY); TEST_UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY); - CatalogTracker ct = new CatalogTracker(TEST_UTIL.getConfiguration()); + CatalogTracker ct = new CatalogTracker(TEST_UTIL.getConnection()); ct.start(); try { HRegionLocation regionLocation = t.getRegionLocation("mmm"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index bb38672..8954fce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -104,7 +104,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; /** - * Run tests that use the HBase clients; {@link HTable} and {@link HTablePool}. + * Run tests that use the HBase clients; {@link HTable} and {@link HConnection}. * Sets up the HBase mini cluster once at start and runs through all client tests. * Each creates a table named for the method and does its stuff against that. */ @@ -243,7 +243,7 @@ public class TestFromClientSide { // Then a ZooKeeperKeepAliveConnection ConnectionManager.HConnectionImplementation connection1 = (ConnectionManager.HConnectionImplementation) - HConnectionManager.getConnection(newConfig); + HConnectionManager.createConnection(newConfig); ZooKeeperKeepAliveConnection z1 = connection1.getKeepAliveZooKeeperWatcher(); z1.getRecoverableZooKeeper().getZooKeeper().exists("/z1", false); @@ -265,7 +265,7 @@ public class TestFromClientSide { newConfig2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "6789"); ConnectionManager.HConnectionImplementation connection2 = (ConnectionManager.HConnectionImplementation) - HConnectionManager.getConnection(newConfig2); + HConnectionManager.createConnection(newConfig2); assertTrue("connections should be different ", connection1 != connection2); @@ -291,7 +291,7 @@ public class TestFromClientSide { z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false); - HConnectionManager.deleteConnection(newConfig); + connection1.close(); try { z2.getRecoverableZooKeeper().getZooKeeper().exists("/z2", false); assertTrue("We should not have a valid connection for z2", false); @@ -302,7 +302,7 @@ public class TestFromClientSide { // We expect success here. - HConnectionManager.deleteConnection(newConfig2); + connection2.close(); try { z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false); assertTrue("We should not have a valid connection for z4", false); @@ -316,8 +316,7 @@ public class TestFromClientSide { @Test public void testRegionCachePreWarm() throws Exception { LOG.info("Starting testRegionCachePreWarm"); - final TableName TABLENAME = - TableName.valueOf("testCachePrewarm"); + final TableName TABLENAME = TableName.valueOf("testCachePrewarm"); Configuration conf = TEST_UTIL.getConfiguration(); // Set up test table: @@ -397,9 +396,18 @@ public class TestFromClientSide { public void testGetConfiguration() throws Exception { byte[] TABLE = Bytes.toBytes("testGetConfiguration"); byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") }; + TEST_UTIL.createTable(TABLE, FAMILIES); Configuration conf = TEST_UTIL.getConfiguration(); - HTable table = TEST_UTIL.createTable(TABLE, FAMILIES, conf); - assertSame(conf, table.getConfiguration()); + HConnection conn = null; + HTableInterface table = null; + try { + conn = HConnectionManager.createConnection(conf); + table = conn.getTable(TABLE); + assertSame(conf, table.getConfiguration()); + } finally { + if (table != null) table.close(); + if (conn != null) conn.close(); + } } /** @@ -4669,7 +4677,7 @@ public class TestFromClientSide { conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize); HTable table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, - conf, Integer.MAX_VALUE); + Integer.MAX_VALUE); final long ts = EnvironmentEdgeManager.currentTimeMillis(); Get get = new Get(ROW); @@ -4706,7 +4714,7 @@ public class TestFromClientSide { conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize); final HTable table = TEST_UTIL.createTable(tableName, - new byte[][] { FAMILY }, conf, 3); + new byte[][] { FAMILY }, 3); final long ts = EnvironmentEdgeManager.currentTimeMillis(); final Get get = new Get(ROW); @@ -5743,8 +5751,7 @@ public class TestFromClientSide { byte[][] FAMILIES = makeNAscii(FAMILY, 3); byte[][] VALUES = makeN(VALUE, 5); long[] ts = { 1000, 2000, 3000, 4000, 5000 }; - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, - TEST_UTIL.getConfiguration(), 3); + HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, 3); Put put = new Put(ROW); put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java index f1c9607..bea2146 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java @@ -113,8 +113,7 @@ public class TestFromClientSide3 { byte[] row, byte[] family, int nFlushes, int nPuts) throws Exception { // connection needed for poll-wait - HConnection conn = HConnectionManager.getConnection(TEST_UTIL - .getConfiguration()); + HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); HRegionLocation loc = table.getRegionLocation(row, true); AdminProtos.AdminService.BlockingInterface server = conn.getAdmin(loc.getServerName()); byte[] regName = loc.getRegionInfo().getRegionName(); @@ -133,6 +132,7 @@ public class TestFromClientSide3 { Thread.sleep(40); } } + conn.close(); } // override the config settings at the CF level and ensure priority @@ -154,8 +154,7 @@ public class TestFromClientSide3 { TableName.valueOf(tableName); HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); - HConnection connection = HConnectionManager.getConnection(TEST_UTIL - .getConfiguration()); + HConnection connection = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); // Create 3 store files. byte[] row = Bytes.toBytes(random.nextInt()); @@ -265,6 +264,7 @@ public class TestFromClientSide3 { LOG.info("alter status finished"); assertNull(hTable.getTableDescriptor().getFamily(FAMILY).getValue( "hbase.hstore.compaction.min")); + connection.close(); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java index ea92bd5..e5f2ac5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java @@ -27,16 +27,13 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.SmallTests; -import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; import org.mortbay.log.Log; -import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; @@ -47,13 +44,10 @@ public class TestHBaseAdminNoCluster { * Verify that PleaseHoldException gets retried. * HBASE-8764 * @throws IOException - * @throws ZooKeeperConnectionException - * @throws MasterNotRunningException - * @throws ServiceException + * @throws ServiceException */ @Test - public void testMasterMonitorCollableRetries() - throws MasterNotRunningException, ZooKeeperConnectionException, IOException, ServiceException { + public void testMasterMonitorCollableRetries() throws IOException, ServiceException { Configuration configuration = HBaseConfiguration.create(); // Set the pause and retry count way down. configuration.setLong(HConstants.HBASE_CLIENT_PAUSE, 1); @@ -71,7 +65,7 @@ public class TestHBaseAdminNoCluster { thenThrow(new ServiceException("Test fail").initCause(new PleaseHoldException("test"))); Mockito.when(connection.getKeepAliveMasterService()).thenReturn(masterAdmin); // Mock up our admin Interfaces - HBaseAdmin admin = new HBaseAdmin(configuration); + HBaseAdmin admin = new HBaseAdmin(connection); try { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testMasterMonitorCollableRetries")); @@ -87,7 +81,7 @@ public class TestHBaseAdminNoCluster { (CreateTableRequest)Mockito.any()); } finally { admin.close(); - if (connection != null)HConnectionManager.deleteConnection(configuration); + if (connection != null) connection.close(); } } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java index 115a49e..f2b4c28 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java @@ -29,7 +29,6 @@ import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.net.SocketTimeoutException; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -73,6 +72,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; +import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.AfterClass; @@ -135,11 +135,6 @@ public class TestHCM { TEST_UTIL.shutdownMiniCluster(); } - - private static int getHConnectionManagerCacheSize(){ - return HConnectionTestingUtility.getConnectionCount(); - } - @Test public void testClusterConnection() throws IOException { ThreadPoolExecutor otherPool = new ThreadPoolExecutor(1, 1, @@ -281,7 +276,7 @@ public class TestHCM { public void testOperationTimeout() throws Exception { HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testOperationTimeout"); hdt.addCoprocessor(SleepAndFailFirstTime.class.getName()); - HTable table = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM}, TEST_UTIL.getConfiguration()); + HTable table = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM}); // Check that it works if the timeout is big enough table.setOperationTimeout(120 * 1000); @@ -494,7 +489,6 @@ public class TestHCM { } finally { syncBlockingFilter.set(true); t.join(); - HConnectionManager.getConnection(c2).close(); TEST_UTIL.getHBaseAdmin().setBalancerRunning(previousBalance, true); } @@ -522,28 +516,6 @@ public class TestHCM { } } - @Test - public void abortingHConnectionRemovesItselfFromHCM() throws Exception { - // Save off current HConnections - Map oldHBaseInstances = - new HashMap(); - oldHBaseInstances.putAll(ConnectionManager.CONNECTION_INSTANCES); - - ConnectionManager.CONNECTION_INSTANCES.clear(); - - try { - HConnection connection = HConnectionManager.getConnection(TEST_UTIL.getConfiguration()); - connection.abort("test abortingHConnectionRemovesItselfFromHCM", new Exception( - "test abortingHConnectionRemovesItselfFromHCM")); - Assert.assertNotSame(connection, - HConnectionManager.getConnection(TEST_UTIL.getConfiguration())); - } finally { - // Put original HConnections back - ConnectionManager.CONNECTION_INSTANCES.clear(); - ConnectionManager.CONNECTION_INSTANCES.putAll(oldHBaseInstances); - } - } - /** * Test that when we delete a location using the first row of a region * that we really delete it. @@ -733,16 +705,23 @@ public class TestHCM { @Test public void testConnectionManagement() throws Exception{ TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM); - HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); - HTableInterface table = conn.getTable(TABLE_NAME1.getName()); - table.close(); - assertFalse(conn.isClosed()); - assertFalse(((HTable)table).getPool().isShutdown()); - table = conn.getTable(TABLE_NAME1.getName()); - table.close(); - assertFalse(((HTable)table).getPool().isShutdown()); - conn.close(); - assertTrue(((HTable)table).getPool().isShutdown()); + HConnection conn = null; + HTableInterface table = null; + try { + conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); + table = conn.getTable(TABLE_NAME1.getName()); + table.close(); + assertFalse(conn.isClosed()); + assertFalse(((HTable)table).getPool().isShutdown()); + table = conn.getTable(TABLE_NAME1.getName()); + table.close(); + assertFalse(((HTable)table).getPool().isShutdown()); + conn.close(); + assertTrue(((HTable)table).getPool().isShutdown()); + } finally { + if (table != null) table.close(); + if (conn != null) conn.close(); + } } /** @@ -766,14 +745,14 @@ public class TestHCM { // Same server as already in cache reporting - overwrites any value despite seqNum. int nextPort = location.getPort() + 1; conn.updateCachedLocation(location.getRegionInfo(), location.getServerName(), - ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1); + ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1); location = conn.getCachedLocation(TABLE_NAME2, ROW); Assert.assertEquals(nextPort, location.getPort()); // No source specified - same. nextPort = location.getPort() + 1; conn.updateCachedLocation(location.getRegionInfo(), location.getServerName(), - ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1); + ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1); location = conn.getCachedLocation(TABLE_NAME2, ROW); Assert.assertEquals(nextPort, location.getPort()); @@ -787,128 +766,12 @@ public class TestHCM { // Lower seqNum - does not overwrite higher seqNum. nextPort = location.getPort() + 1; conn.updateCachedLocation(location.getRegionInfo(), anySource, - ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1); + ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1); location = conn.getCachedLocation(TABLE_NAME2, ROW); Assert.assertEquals(nextPort - 1, location.getPort()); } /** - * Make sure that {@link Configuration} instances that are essentially the - * same map to the same {@link HConnection} instance. - */ - @Test - public void testConnectionSameness() throws Exception { - HConnection previousConnection = null; - for (int i = 0; i < 2; i++) { - // set random key to differentiate the connection from previous ones - Configuration configuration = TEST_UTIL.getConfiguration(); - configuration.set("some_key", String.valueOf(_randy.nextInt())); - LOG.info("The hash code of the current configuration is: " - + configuration.hashCode()); - HConnection currentConnection = HConnectionManager - .getConnection(configuration); - if (previousConnection != null) { - assertTrue( - "Did not get the same connection even though its key didn't change", - previousConnection == currentConnection); - } - previousConnection = currentConnection; - // change the configuration, so that it is no longer reachable from the - // client's perspective. However, since its part of the LRU doubly linked - // list, it will eventually get thrown out, at which time it should also - // close the corresponding {@link HConnection}. - configuration.set("other_key", String.valueOf(_randy.nextInt())); - } - } - - /** - * Makes sure that there is no leaking of - * {@link ConnectionManager.HConnectionImplementation} in the {@link HConnectionManager} - * class. - */ - @Test - public void testConnectionUniqueness() throws Exception { - int zkmaxconnections = TEST_UTIL.getConfiguration(). - getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, - HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS); - // Test up to a max that is < the maximum number of zk connections. If we - // go above zk connections, we just fall into cycle where we are failing - // to set up a session and test runs for a long time. - int maxConnections = Math.min(zkmaxconnections - 1, 20); - List connections = new ArrayList(maxConnections); - HConnection previousConnection = null; - try { - for (int i = 0; i < maxConnections; i++) { - // set random key to differentiate the connection from previous ones - Configuration configuration = new Configuration(TEST_UTIL.getConfiguration()); - configuration.set("some_key", String.valueOf(_randy.nextInt())); - configuration.set(HConstants.HBASE_CLIENT_INSTANCE_ID, - String.valueOf(_randy.nextInt())); - LOG.info("The hash code of the current configuration is: " - + configuration.hashCode()); - HConnection currentConnection = - HConnectionManager.getConnection(configuration); - if (previousConnection != null) { - assertTrue("Got the same connection even though its key changed!", - previousConnection != currentConnection); - } - // change the configuration, so that it is no longer reachable from the - // client's perspective. However, since its part of the LRU doubly linked - // list, it will eventually get thrown out, at which time it should also - // close the corresponding {@link HConnection}. - configuration.set("other_key", String.valueOf(_randy.nextInt())); - - previousConnection = currentConnection; - LOG.info("The current HConnectionManager#HBASE_INSTANCES cache size is: " - + getHConnectionManagerCacheSize()); - Thread.sleep(50); - connections.add(currentConnection); - } - } finally { - for (HConnection c: connections) { - // Clean up connections made so we don't interfere w/ subsequent tests. - HConnectionManager.deleteConnection(c.getConfiguration()); - } - } - } - - @Test - public void testClosing() throws Exception { - Configuration configuration = - new Configuration(TEST_UTIL.getConfiguration()); - configuration.set(HConstants.HBASE_CLIENT_INSTANCE_ID, - String.valueOf(_randy.nextInt())); - - HConnection c1 = HConnectionManager.createConnection(configuration); - // We create two connections with the same key. - HConnection c2 = HConnectionManager.createConnection(configuration); - - HConnection c3 = HConnectionManager.getConnection(configuration); - HConnection c4 = HConnectionManager.getConnection(configuration); - assertTrue(c3 == c4); - - c1.close(); - assertTrue(c1.isClosed()); - assertFalse(c2.isClosed()); - assertFalse(c3.isClosed()); - - c3.close(); - // still a reference left - assertFalse(c3.isClosed()); - c3.close(); - assertTrue(c3.isClosed()); - // c3 was removed from the cache - HConnection c5 = HConnectionManager.getConnection(configuration); - assertTrue(c5 != c3); - - assertFalse(c2.isClosed()); - c2.close(); - assertTrue(c2.isClosed()); - c5.close(); - assertTrue(c5.isClosed()); - } - - /** * Trivial test to verify that nobody messes with * {@link HConnectionManager#createConnection(Configuration)} */ @@ -920,10 +783,6 @@ public class TestHCM { // created from the same configuration, yet they are different assertTrue(c1 != c2); assertTrue(c1.getConfiguration() == c2.getConfiguration()); - // make sure these were not cached - HConnection c3 = HConnectionManager.getConnection(configuration); - assertTrue(c1 != c3); - assertTrue(c2 != c3); } @@ -942,13 +801,13 @@ public class TestHCM { TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_CLIENT_PORT)); // This should be enough to connect - HConnection conn = HConnectionManager.getConnection(c); + HConnection conn = HConnectionManager.createConnection(c); assertTrue( conn.isMasterRunning() ); conn.close(); } private int setNumTries(HConnectionImplementation hci, int newVal) throws Exception { - Field numTries = hci.getClass().getDeclaredField("numTries"); + Field numTries = ReflectionUtils.getAllDeclaredField(hci.getClass(), "numTries"); numTries.setAccessible(true); Field modifiersField = Field.class.getDeclaredField("modifiers"); modifiersField.setAccessible(true); @@ -964,8 +823,7 @@ public class TestHCM { HTable table = TEST_UTIL.createTable(TABLE_NAME3, FAM_NAM); TEST_UTIL.createMultiRegions(table, FAM_NAM); ConnectionManager.HConnectionImplementation conn = - (ConnectionManager.HConnectionImplementation) - HConnectionManager.getConnection(TEST_UTIL.getConfiguration()); + (ConnectionManager.HConnectionImplementation) TEST_UTIL.getConnection(); // We're now going to move the region and check that it works for the client // First a new put to add the location in the cache @@ -986,6 +844,7 @@ public class TestHCM { // Now moving the region to the second server HRegionLocation toMove = conn.getCachedLocation(TABLE_NAME3, ROW_X); + assertNotNull(toMove); byte[] regionName = toMove.getRegionInfo().getRegionName(); byte[] encodedRegionNameBytes = toMove.getRegionInfo().getEncodedNameAsBytes(); @@ -1165,9 +1024,9 @@ public class TestHCM { public void run() { while (!Thread.interrupted()) { try { - HConnection conn = HConnectionManager.getConnection(config); + HConnection conn = HConnectionManager.createConnection(config); LOG.info("Connection " + conn); - HConnectionManager.deleteStaleConnection(conn); + conn.close(); LOG.info("Connection closed " + conn); // TODO: This sleep time should be less than the time that it takes to open and close // a table. Ideally we would do a few runs first to measure. For now this is @@ -1183,7 +1042,7 @@ public class TestHCM { for (int i = 0; i < 30; i++) { HConnection c1 = null; try { - c1 = ConnectionManager.getConnectionInternal(config); + c1 = ConnectionManager.createConnection(config); LOG.info("HTable connection " + i + " " + c1); HTable table = new HTable(config, TABLE_NAME4, pool); table.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java index 2f0bf37..1ce901d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java @@ -72,7 +72,7 @@ public class TestHTableMultiplexer { List failedPuts; boolean success; - HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(), + HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConnection(), PER_REGIONSERVER_QUEUE_SIZE); HTable ht = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }, VERSION, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java deleted file mode 100644 index 322ebaa..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java +++ /dev/null @@ -1,364 +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 org.apache.hadoop.hbase.*; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.PoolMap.PoolType; -import org.junit.*; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Suite; - -/** - * Tests HTablePool. - */ -@RunWith(Suite.class) -@Suite.SuiteClasses({TestHTablePool.TestHTableReusablePool.class, TestHTablePool.TestHTableThreadLocalPool.class}) -@Category(MediumTests.class) -public class TestHTablePool { - private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private final static byte[] TABLENAME = Bytes.toBytes("TestHTablePool"); - - public abstract static class TestHTablePoolType { - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TEST_UTIL.startMiniCluster(1); - TEST_UTIL.createTable(TABLENAME, HConstants.CATALOG_FAMILY); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - protected abstract PoolType getPoolType(); - - @Test - public void testTableWithStringName() throws Exception { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), - Integer.MAX_VALUE, getPoolType()); - String tableName = Bytes.toString(TABLENAME); - - // Request a table from an empty pool - HTableInterface table = pool.getTable(tableName); - Assert.assertNotNull(table); - - // Close table (returns table to the pool) - table.close(); - - // Request a table of the same name - HTableInterface sameTable = pool.getTable(tableName); - Assert.assertSame( - ((HTablePool.PooledHTable) table).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable).getWrappedTable()); - } - - @Test - public void testTableWithByteArrayName() throws IOException { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), - Integer.MAX_VALUE, getPoolType()); - - // Request a table from an empty pool - HTableInterface table = pool.getTable(TABLENAME); - Assert.assertNotNull(table); - - // Close table (returns table to the pool) - table.close(); - - // Request a table of the same name - HTableInterface sameTable = pool.getTable(TABLENAME); - Assert.assertSame( - ((HTablePool.PooledHTable) table).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable).getWrappedTable()); - } - - @Test - public void testTablesWithDifferentNames() throws IOException { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), - Integer.MAX_VALUE, getPoolType()); - // We add the class to the table name as the HBase cluster is reused - // during the tests: this gives naming unicity. - byte[] otherTable = Bytes.toBytes( - "OtherTable_" + getClass().getSimpleName() - ); - TEST_UTIL.createTable(otherTable, HConstants.CATALOG_FAMILY); - - // Request a table from an empty pool - HTableInterface table1 = pool.getTable(TABLENAME); - HTableInterface table2 = pool.getTable(otherTable); - Assert.assertNotNull(table2); - - // Close tables (returns tables to the pool) - table1.close(); - table2.close(); - - // Request tables of the same names - HTableInterface sameTable1 = pool.getTable(TABLENAME); - HTableInterface sameTable2 = pool.getTable(otherTable); - Assert.assertSame( - ((HTablePool.PooledHTable) table1).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable1).getWrappedTable()); - Assert.assertSame( - ((HTablePool.PooledHTable) table2).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable2).getWrappedTable()); - } - @Test - public void testProxyImplementationReturned() { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), - Integer.MAX_VALUE); - String tableName = Bytes.toString(TABLENAME);// Request a table from - // an - // empty pool - HTableInterface table = pool.getTable(tableName); - - // Test if proxy implementation is returned - Assert.assertTrue(table instanceof HTablePool.PooledHTable); - } - - @Test - public void testDeprecatedUsagePattern() throws IOException { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), - Integer.MAX_VALUE); - String tableName = Bytes.toString(TABLENAME);// Request a table from - // an - // empty pool - - // get table will return proxy implementation - HTableInterface table = pool.getTable(tableName); - - // put back the proxy implementation instead of closing it - pool.putTable(table); - - // Request a table of the same name - HTableInterface sameTable = pool.getTable(tableName); - - // test no proxy over proxy created - Assert.assertSame(((HTablePool.PooledHTable) table).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable).getWrappedTable()); - } - - @Test - public void testReturnDifferentTable() throws IOException { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), - Integer.MAX_VALUE); - String tableName = Bytes.toString(TABLENAME);// Request a table from - // an - // empty pool - - // get table will return proxy implementation - final HTableInterface table = pool.getTable(tableName); - HTableInterface alienTable = new HTable(TEST_UTIL.getConfiguration(), - TABLENAME) { - // implementation doesn't matter as long the table is not from - // pool - }; - try { - // put the wrong table in pool - pool.putTable(alienTable); - Assert.fail("alien table accepted in pool"); - } catch (IllegalArgumentException e) { - Assert.assertTrue("alien table rejected", true); - } - } - - @Test - public void testHTablePoolCloseTwice() throws Exception { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), - Integer.MAX_VALUE, getPoolType()); - String tableName = Bytes.toString(TABLENAME); - - // Request a table from an empty pool - HTableInterface table = pool.getTable(tableName); - Assert.assertNotNull(table); - Assert.assertTrue(((HTablePool.PooledHTable) table).isOpen()); - // Close table (returns table to the pool) - table.close(); - // check if the table is closed - Assert.assertFalse(((HTablePool.PooledHTable) table).isOpen()); - try { - table.close(); - Assert.fail("Should not allow table to be closed twice"); - } catch (IllegalStateException ex) { - Assert.assertTrue("table cannot be closed twice", true); - } finally { - pool.close(); - } - - } - - - - } - - @Category(MediumTests.class) - public static class TestHTableReusablePool extends TestHTablePoolType { - @Override - protected PoolType getPoolType() { - return PoolType.Reusable; - } - - @Test - public void testTableWithMaxSize() throws Exception { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), 2, - getPoolType()); - - // Request tables from an empty pool - HTableInterface table1 = pool.getTable(TABLENAME); - HTableInterface table2 = pool.getTable(TABLENAME); - HTableInterface table3 = pool.getTable(TABLENAME); - - // Close tables (returns tables to the pool) - table1.close(); - table2.close(); - // The pool should reject this one since it is already full - table3.close(); - - // Request tables of the same name - HTableInterface sameTable1 = pool.getTable(TABLENAME); - HTableInterface sameTable2 = pool.getTable(TABLENAME); - HTableInterface sameTable3 = pool.getTable(TABLENAME); - Assert.assertSame( - ((HTablePool.PooledHTable) table1).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable1).getWrappedTable()); - Assert.assertSame( - ((HTablePool.PooledHTable) table2).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable2).getWrappedTable()); - Assert.assertNotSame( - ((HTablePool.PooledHTable) table3).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable3).getWrappedTable()); - } - - @Test - public void testCloseTablePool() throws IOException { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), 4, - getPoolType()); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); - - if (admin.tableExists(TABLENAME)) { - admin.disableTable(TABLENAME); - admin.deleteTable(TABLENAME); - } - - HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(TABLENAME)); - tableDescriptor.addFamily(new HColumnDescriptor("randomFamily")); - admin.createTable(tableDescriptor); - - // Request tables from an empty pool - HTableInterface[] tables = new HTableInterface[4]; - for (int i = 0; i < 4; ++i) { - tables[i] = pool.getTable(TABLENAME); - } - - pool.closeTablePool(TABLENAME); - - for (int i = 0; i < 4; ++i) { - tables[i].close(); - } - - Assert.assertEquals(4, - pool.getCurrentPoolSize(Bytes.toString(TABLENAME))); - - pool.closeTablePool(TABLENAME); - - Assert.assertEquals(0, - pool.getCurrentPoolSize(Bytes.toString(TABLENAME))); - } - } - - @Category(MediumTests.class) - public static class TestHTableThreadLocalPool extends TestHTablePoolType { - @Override - protected PoolType getPoolType() { - return PoolType.ThreadLocal; - } - - @Test - public void testTableWithMaxSize() throws Exception { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), 2, - getPoolType()); - - // Request tables from an empty pool - HTableInterface table1 = pool.getTable(TABLENAME); - HTableInterface table2 = pool.getTable(TABLENAME); - HTableInterface table3 = pool.getTable(TABLENAME); - - // Close tables (returns tables to the pool) - table1.close(); - table2.close(); - // The pool should not reject this one since the number of threads - // <= 2 - table3.close(); - - // Request tables of the same name - HTableInterface sameTable1 = pool.getTable(TABLENAME); - HTableInterface sameTable2 = pool.getTable(TABLENAME); - HTableInterface sameTable3 = pool.getTable(TABLENAME); - Assert.assertSame( - ((HTablePool.PooledHTable) table3).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable1).getWrappedTable()); - Assert.assertSame( - ((HTablePool.PooledHTable) table3).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable2).getWrappedTable()); - Assert.assertSame( - ((HTablePool.PooledHTable) table3).getWrappedTable(), - ((HTablePool.PooledHTable) sameTable3).getWrappedTable()); - } - - @Test - public void testCloseTablePool() throws IOException { - HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), 4, - getPoolType()); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); - - if (admin.tableExists(TABLENAME)) { - admin.disableTable(TABLENAME); - admin.deleteTable(TABLENAME); - } - - HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(TABLENAME)); - tableDescriptor.addFamily(new HColumnDescriptor("randomFamily")); - admin.createTable(tableDescriptor); - - // Request tables from an empty pool - HTableInterface[] tables = new HTableInterface[4]; - for (int i = 0; i < 4; ++i) { - tables[i] = pool.getTable(TABLENAME); - } - - pool.closeTablePool(TABLENAME); - - for (int i = 0; i < 4; ++i) { - tables[i].close(); - } - - Assert.assertEquals(1, - pool.getCurrentPoolSize(Bytes.toString(TABLENAME))); - - pool.closeTablePool(TABLENAME); - - Assert.assertEquals(0, - pool.getCurrentPoolSize(Bytes.toString(TABLENAME))); - } - } - -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java index 4e5a656..401078c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java @@ -87,27 +87,30 @@ public class TestMetaScanner { doReturn(true).when(visitor).processRow((Result)anyObject()); // Scanning the entire table should give us three rows - MetaScanner.metaScan(conf, null, visitor, TABLENAME); + MetaScanner.metaScan(TEST_UTIL.getConnection(), visitor, TABLENAME); verify(visitor, times(3)).processRow((Result)anyObject()); // Scanning the table with a specified empty start row should also // give us three hbase:meta rows reset(visitor); doReturn(true).when(visitor).processRow((Result)anyObject()); - MetaScanner.metaScan(conf, visitor, TABLENAME, HConstants.EMPTY_BYTE_ARRAY, 1000); + MetaScanner.metaScan(TEST_UTIL.getConnection(), visitor, TABLENAME, + HConstants.EMPTY_BYTE_ARRAY, 1000, TableName.META_TABLE_NAME); verify(visitor, times(3)).processRow((Result)anyObject()); // Scanning the table starting in the middle should give us two rows: // region_a and region_b reset(visitor); doReturn(true).when(visitor).processRow((Result)anyObject()); - MetaScanner.metaScan(conf, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1000); + MetaScanner.metaScan(TEST_UTIL.getConnection(), visitor, TABLENAME, + Bytes.toBytes("region_ac"), 1000, TableName.META_TABLE_NAME); verify(visitor, times(2)).processRow((Result)anyObject()); // Scanning with a limit of 1 should only give us one row reset(visitor); doReturn(true).when(visitor).processRow((Result)anyObject()); - MetaScanner.metaScan(conf, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1); + MetaScanner.metaScan(TEST_UTIL.getConnection(), visitor, TABLENAME, + Bytes.toBytes("region_ac"), 1, TableName.META_TABLE_NAME); verify(visitor, times(1)).processRow((Result)anyObject()); table.close(); } @@ -138,7 +141,7 @@ public class TestMetaScanner { while (!isStopped()) { try { List regions = MetaScanner.listAllRegions( - TEST_UTIL.getConfiguration(), false); + TEST_UTIL.getConnection(), false); //select a random region HRegionInfo parent = regions.get(random.nextInt(regions.size())); @@ -191,7 +194,7 @@ public class TestMetaScanner { while(!isStopped()) { try { NavigableMap regions = - MetaScanner.allTableRegions(TEST_UTIL.getConfiguration(), null, TABLENAME, false); + MetaScanner.allTableRegions(TEST_UTIL.getConnection(), TABLENAME, false); LOG.info("-------"); byte[] lastEndKey = HConstants.EMPTY_START_ROW; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java index cc19937..a89e5cf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java @@ -95,7 +95,7 @@ public class TestMultiParallel { // Wait until completing balance UTIL.waitFor(15 * 1000, UTIL.predicateNoRegionsInTransition()); } - HConnection conn = HConnectionManager.getConnection(UTIL.getConfiguration()); + HConnection conn = HConnectionManager.createConnection(UTIL.getConfiguration()); conn.clearRegionCache(); conn.close(); LOG.info("before done"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java index 296bea6..04c07da 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java @@ -166,7 +166,7 @@ public class TestRestoreSnapshotFromClient { public void testRestoreSchemaChange() throws Exception { byte[] TEST_FAMILY2 = Bytes.toBytes("cf2"); - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + HTable table = (HTable) TEST_UTIL.getConnection().getTable(tableName); // Add one column family and put some data in it admin.disableTable(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java index d358a13..afa5688 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java @@ -40,6 +40,8 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import java.io.IOException; + /** * A test class to cover aggregate functions, that can be implemented using * Coprocessors. @@ -64,7 +66,6 @@ public class TestAggregateProtocol { private static byte[][] ROWS = makeN(ROW, ROWSIZE); private static HBaseTestingUtility util = new HBaseTestingUtility(); - private static Configuration conf = util.getConfiguration(); /** * A set up method to start the test cluster. AggregateProtocolImpl is @@ -74,8 +75,8 @@ public class TestAggregateProtocol { @BeforeClass public static void setupBeforeClass() throws Exception { - conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, - "org.apache.hadoop.hbase.coprocessor.AggregateImplementation"); + util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, + "org.apache.hadoop.hbase.coprocessor.AggregateImplementation"); util.startMiniCluster(2); HTable table = util.createTable(TEST_TABLE, TEST_FAMILY); @@ -132,7 +133,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testMedianWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY,TEST_QUALIFIER); final ColumnInterpreter ci = @@ -153,7 +154,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testRowCountWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(ROWS[2]); @@ -171,7 +172,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testRowCountAllTable() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); final ColumnInterpreter ci = new LongColumnInterpreter(); @@ -186,8 +187,8 @@ public class TestAggregateProtocol { * @throws Throwable */ @Test (timeout=300000) - public void testRowCountWithInvalidRange1() { - AggregationClient aClient = new AggregationClient(conf); + public void testRowCountWithInvalidRange1() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.setStartRow(ROWS[5]); scan.setStopRow(ROWS[2]); @@ -210,8 +211,8 @@ public class TestAggregateProtocol { * @throws Throwable */ @Test (timeout=300000) - public void testRowCountWithInvalidRange2() { - AggregationClient aClient = new AggregationClient(conf); + public void testRowCountWithInvalidRange2() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.setStartRow(ROWS[5]); scan.setStopRow(ROWS[5]); @@ -229,7 +230,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testRowCountWithNullCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); final ColumnInterpreter ci = @@ -241,7 +242,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testRowCountWithPrefixFilter() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); final ColumnInterpreter ci = @@ -263,7 +264,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testMaxWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); final ColumnInterpreter ci = @@ -277,7 +278,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testMaxWithValidRange2() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -290,7 +291,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testMaxWithValidRangeWithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); final ColumnInterpreter ci = @@ -301,7 +302,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testMaxWithValidRange2WithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -313,8 +314,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testMaxWithValidRangeWithNullCF() { - AggregationClient aClient = new AggregationClient(conf); + public void testMaxWithValidRangeWithNullCF() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); final ColumnInterpreter ci = new LongColumnInterpreter(); Scan scan = new Scan(); @@ -329,8 +330,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testMaxWithInvalidRange() { - AggregationClient aClient = new AggregationClient(conf); + public void testMaxWithInvalidRange() throws IOException{ + AggregationClient aClient = new AggregationClient(util.getConnection()); final ColumnInterpreter ci = new LongColumnInterpreter(); Scan scan = new Scan(); @@ -354,7 +355,7 @@ public class TestAggregateProtocol { scan.setStartRow(ROWS[4]); scan.setStopRow(ROWS[4]); try { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); final ColumnInterpreter ci = new LongColumnInterpreter(); max = aClient.max(TEST_TABLE, ci, scan); @@ -367,7 +368,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testMaxWithFilter() throws Throwable { Long max = 0l; - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); Filter f = new PrefixFilter(Bytes.toBytes("foo:bar")); @@ -387,7 +388,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testMinWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(HConstants.EMPTY_START_ROW); @@ -404,7 +405,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testMinWithValidRange2() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -417,7 +418,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testMinWithValidRangeWithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(HConstants.EMPTY_START_ROW); @@ -431,7 +432,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testMinWithValidRange2WithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -443,8 +444,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testMinWithValidRangeWithNullCF() { - AggregationClient aClient = new AggregationClient(conf); + public void testMinWithValidRangeWithNullCF() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.setStartRow(ROWS[5]); scan.setStopRow(ROWS[15]); @@ -460,8 +461,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testMinWithInvalidRange() { - AggregationClient aClient = new AggregationClient(conf); + public void testMinWithInvalidRange() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Long min = null; Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); @@ -477,8 +478,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testMinWithInvalidRange2() { - AggregationClient aClient = new AggregationClient(conf); + public void testMinWithInvalidRange2() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -495,7 +496,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testMinWithFilter() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); Filter f = new PrefixFilter(Bytes.toBytes("foo:bar")); @@ -515,7 +516,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testSumWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY,TEST_QUALIFIER); final ColumnInterpreter ci = @@ -530,7 +531,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testSumWithValidRange2() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY,TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -543,7 +544,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testSumWithValidRangeWithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); final ColumnInterpreter ci = @@ -555,7 +556,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testSumWithValidRange2WithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -567,8 +568,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testSumWithValidRangeWithNullCF() { - AggregationClient aClient = new AggregationClient(conf); + public void testSumWithValidRangeWithNullCF() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.setStartRow(ROWS[6]); scan.setStopRow(ROWS[7]); @@ -584,8 +585,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testSumWithInvalidRange() { - AggregationClient aClient = new AggregationClient(conf); + public void testSumWithInvalidRange() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -602,7 +603,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testSumWithFilter() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Filter f = new PrefixFilter(Bytes.toBytes("foo:bar")); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); @@ -622,7 +623,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testAvgWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY,TEST_QUALIFIER); final ColumnInterpreter ci = @@ -637,7 +638,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testAvgWithValidRange2() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY,TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -650,7 +651,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testAvgWithValidRangeWithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); final ColumnInterpreter ci = @@ -662,7 +663,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testAvgWithValidRange2WithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -674,8 +675,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testAvgWithValidRangeWithNullCF() { - AggregationClient aClient = new AggregationClient(conf); + public void testAvgWithValidRangeWithNullCF() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); final ColumnInterpreter ci = new LongColumnInterpreter(); @@ -689,8 +690,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testAvgWithInvalidRange() { - AggregationClient aClient = new AggregationClient(conf); + public void testAvgWithInvalidRange() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY,TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -707,7 +708,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testAvgWithFilter() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY,TEST_QUALIFIER); Filter f = new PrefixFilter(Bytes.toBytes("foo:bar")); @@ -727,7 +728,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testStdWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY,TEST_QUALIFIER); final ColumnInterpreter ci = @@ -742,7 +743,7 @@ public class TestAggregateProtocol { */ @Test (timeout=300000) public void testStdWithValidRange2() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY,TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -755,7 +756,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testStdWithValidRangeWithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); final ColumnInterpreter ci = @@ -767,7 +768,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testStdWithValidRange2WithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -779,8 +780,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testStdWithValidRangeWithNullCF() { - AggregationClient aClient = new AggregationClient(conf); + public void testStdWithValidRangeWithNullCF() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.setStartRow(ROWS[6]); scan.setStopRow(ROWS[17]); @@ -796,8 +797,8 @@ public class TestAggregateProtocol { } @Test (timeout=300000) - public void testStdWithInvalidRange() { - AggregationClient aClient = new AggregationClient(conf); + public void testStdWithInvalidRange() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -814,7 +815,7 @@ public class TestAggregateProtocol { @Test (timeout=300000) public void testStdWithFilter() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Filter f = new PrefixFilter(Bytes.toBytes("foo:bar")); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java index 06773bd..77c0abb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.coprocessor; import static org.junit.Assert.assertEquals; + +import java.io.IOException; import java.math.BigDecimal; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -129,7 +131,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testMedianWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); final ColumnInterpreter ci = @@ -148,7 +150,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testMaxWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); final ColumnInterpreter ci = @@ -162,7 +164,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testMaxWithValidRange2() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -175,7 +177,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testMaxWithValidRangeWithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); final ColumnInterpreter ci = @@ -186,7 +188,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testMaxWithValidRange2WithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -198,8 +200,8 @@ public class TestBigDecimalColumnInterpreter { } @Test (timeout=300000) - public void testMaxWithValidRangeWithNullCF() { - AggregationClient aClient = new AggregationClient(conf); + public void testMaxWithValidRangeWithNullCF() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); final ColumnInterpreter ci = new BigDecimalColumnInterpreter(); Scan scan = new Scan(); @@ -214,8 +216,8 @@ public class TestBigDecimalColumnInterpreter { } @Test (timeout=300000) - public void testMaxWithInvalidRange() { - AggregationClient aClient = new AggregationClient(conf); + public void testMaxWithInvalidRange() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); final ColumnInterpreter ci = new BigDecimalColumnInterpreter(); Scan scan = new Scan(); @@ -240,7 +242,7 @@ public class TestBigDecimalColumnInterpreter { scan.setStartRow(ROWS[4]); scan.setStopRow(ROWS[4]); try { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); final ColumnInterpreter ci = new BigDecimalColumnInterpreter(); max = aClient.max(TEST_TABLE, ci, scan); @@ -253,7 +255,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testMaxWithFilter() throws Throwable { BigDecimal max = BigDecimal.ZERO; - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); Filter f = new PrefixFilter(Bytes.toBytes("foo:bar")); @@ -273,7 +275,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testMinWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(HConstants.EMPTY_START_ROW); @@ -289,7 +291,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testMinWithValidRange2() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -302,7 +304,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testMinWithValidRangeWithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(HConstants.EMPTY_START_ROW); @@ -315,7 +317,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testMinWithValidRange2WithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -327,8 +329,8 @@ public class TestBigDecimalColumnInterpreter { } @Test (timeout=300000) - public void testMinWithValidRangeWithNullCF() { - AggregationClient aClient = new AggregationClient(conf); + public void testMinWithValidRangeWithNullCF() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.setStartRow(ROWS[5]); scan.setStopRow(ROWS[15]); @@ -344,8 +346,8 @@ public class TestBigDecimalColumnInterpreter { } @Test (timeout=300000) - public void testMinWithInvalidRange() { - AggregationClient aClient = new AggregationClient(conf); + public void testMinWithInvalidRange() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); BigDecimal min = null; Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); @@ -361,8 +363,8 @@ public class TestBigDecimalColumnInterpreter { } @Test (timeout=300000) - public void testMinWithInvalidRange2() { - AggregationClient aClient = new AggregationClient(conf); + public void testMinWithInvalidRange2() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -379,7 +381,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testMinWithFilter() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); Filter f = new PrefixFilter(Bytes.toBytes("foo:bar")); @@ -399,7 +401,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testSumWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); final ColumnInterpreter ci = @@ -413,7 +415,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testSumWithValidRange2() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -426,7 +428,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testSumWithValidRangeWithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); final ColumnInterpreter ci = @@ -437,7 +439,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testSumWithValidRange2WithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -449,8 +451,8 @@ public class TestBigDecimalColumnInterpreter { } @Test (timeout=300000) - public void testSumWithValidRangeWithNullCF() { - AggregationClient aClient = new AggregationClient(conf); + public void testSumWithValidRangeWithNullCF() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.setStartRow(ROWS[6]); scan.setStopRow(ROWS[7]); @@ -466,8 +468,8 @@ public class TestBigDecimalColumnInterpreter { } @Test (timeout=300000) - public void testSumWithInvalidRange() { - AggregationClient aClient = new AggregationClient(conf); + public void testSumWithInvalidRange() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -484,7 +486,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testSumWithFilter() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Filter f = new PrefixFilter(Bytes.toBytes("foo:bar")); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); @@ -504,7 +506,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testAvgWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); final ColumnInterpreter ci = @@ -518,7 +520,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testAvgWithValidRange2() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -531,7 +533,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testAvgWithValidRangeWithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); final ColumnInterpreter ci = @@ -542,7 +544,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testAvgWithValidRange2WithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -554,8 +556,8 @@ public class TestBigDecimalColumnInterpreter { } @Test (timeout=300000) - public void testAvgWithValidRangeWithNullCF() { - AggregationClient aClient = new AggregationClient(conf); + public void testAvgWithValidRangeWithNullCF() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); final ColumnInterpreter ci = new BigDecimalColumnInterpreter(); @@ -569,8 +571,8 @@ public class TestBigDecimalColumnInterpreter { } @Test (timeout=300000) - public void testAvgWithInvalidRange() { - AggregationClient aClient = new AggregationClient(conf); + public void testAvgWithInvalidRange() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -587,7 +589,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testAvgWithFilter() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); Filter f = new PrefixFilter(Bytes.toBytes("foo:bar")); @@ -607,7 +609,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testStdWithValidRange() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); final ColumnInterpreter ci = @@ -622,7 +624,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testStdWithValidRange2() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addColumn(TEST_FAMILY, TEST_QUALIFIER); scan.setStartRow(ROWS[5]); @@ -639,7 +641,7 @@ public class TestBigDecimalColumnInterpreter { */ @Test (timeout=300000) public void testStdWithValidRangeWithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); final ColumnInterpreter ci = @@ -650,7 +652,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testStdWithValidRange2WithNoCQ() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -663,8 +665,8 @@ public class TestBigDecimalColumnInterpreter { } @Test (timeout=300000) - public void testStdWithValidRangeWithNullCF() { - AggregationClient aClient = new AggregationClient(conf); + public void testStdWithValidRangeWithNullCF() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.setStartRow(ROWS[6]); scan.setStopRow(ROWS[17]); @@ -680,8 +682,8 @@ public class TestBigDecimalColumnInterpreter { } @Test - public void testStdWithInvalidRange() { - AggregationClient aClient = new AggregationClient(conf); + public void testStdWithInvalidRange() throws IOException { + AggregationClient aClient = new AggregationClient(util.getConnection()); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); scan.setStartRow(ROWS[6]); @@ -698,7 +700,7 @@ public class TestBigDecimalColumnInterpreter { @Test (timeout=300000) public void testStdWithFilter() throws Throwable { - AggregationClient aClient = new AggregationClient(conf); + AggregationClient aClient = new AggregationClient(util.getConnection()); Filter f = new PrefixFilter(Bytes.toBytes("foo:bar")); Scan scan = new Scan(); scan.addFamily(TEST_FAMILY); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java index 99fc48d..18304f7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java @@ -140,15 +140,15 @@ public class TestTableMapReduceUtil { Assert.assertNotNull(presidentsTable); Configuration cfg = UTIL.getConfiguration(); JobConf jobConf = new JobConf(cfg); - TableMapReduceUtil.setNumReduceTasks(TABLE_NAME, jobConf); - TableMapReduceUtil.limitNumReduceTasks(TABLE_NAME, jobConf); + TableMapReduceUtil.setNumReduceTasks(UTIL.getConnection(), TABLE_NAME, jobConf); + TableMapReduceUtil.limitNumReduceTasks(UTIL.getConnection(), TABLE_NAME, jobConf); TableMapReduceUtil.setScannerCaching(jobConf, 100); assertEquals(1, jobConf.getNumReduceTasks()); assertEquals(100, jobConf.getInt("hbase.client.scanner.caching", 0)); jobConf.setNumReduceTasks(10); - TableMapReduceUtil.setNumMapTasks(TABLE_NAME, jobConf); - TableMapReduceUtil.limitNumReduceTasks(TABLE_NAME, jobConf); + TableMapReduceUtil.setNumMapTasks(UTIL.getConnection(), TABLE_NAME, jobConf); + TableMapReduceUtil.limitNumReduceTasks(UTIL.getConnection(), TABLE_NAME, jobConf); assertEquals(1, jobConf.getNumReduceTasks()); } @@ -158,13 +158,13 @@ public class TestTableMapReduceUtil { throws IOException { Configuration cfg = UTIL.getConfiguration(); JobConf jobConf = new JobConf(cfg); - TableMapReduceUtil.setNumReduceTasks(TABLE_NAME, jobConf); - TableMapReduceUtil.limitNumMapTasks(TABLE_NAME, jobConf); + TableMapReduceUtil.setNumReduceTasks(UTIL.getConnection(), TABLE_NAME, jobConf); + TableMapReduceUtil.limitNumMapTasks(UTIL.getConnection(), TABLE_NAME, jobConf); assertEquals(1, jobConf.getNumMapTasks()); jobConf.setNumMapTasks(10); - TableMapReduceUtil.setNumMapTasks(TABLE_NAME, jobConf); - TableMapReduceUtil.limitNumMapTasks(TABLE_NAME, jobConf); + TableMapReduceUtil.setNumMapTasks(UTIL.getConnection(), TABLE_NAME, jobConf); + TableMapReduceUtil.limitNumMapTasks(UTIL.getConnection(), TABLE_NAME, jobConf); assertEquals(1, jobConf.getNumMapTasks()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java index 077c95a..0e6ec56 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LargeTests; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -114,7 +113,6 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { // Wait for the labels table to become available util.waitTableEnabled(VisibilityConstants.LABELS_TABLE_NAME.getName(), 50000); createLabels(); - HBaseAdmin admin = new HBaseAdmin(util.getConfiguration()); util.startMiniMapReduceCluster(); } @@ -124,7 +122,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { public VisibilityLabelsResponse run() throws Exception { String[] labels = { SECRET, TOPSECRET, CONFIDENTIAL, PUBLIC, PRIVATE }; try { - VisibilityClient.addLabels(conf, labels); + VisibilityClient.addLabels(util.getConnection(), labels); LOG.info("Added labels "); } catch (Throwable t) { LOG.error("Error in adding labels" , t); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java index 4a06b6d..925a550 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java @@ -48,7 +48,9 @@ import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.catalog.MetaMockingUtil; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; +import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.ExecutorService; @@ -96,8 +98,9 @@ import com.google.protobuf.ServiceException; /** * Test {@link AssignmentManager} */ +// TODO: fix mocking META necessary for test @Category(MediumTests.class) -public class TestAssignmentManager { +public abstract class TestAssignmentManager { private static final HBaseTestingUtility HTU = new HBaseTestingUtility(); private static final ServerName SERVERNAME_A = ServerName.valueOf("example.org", 1234, 5678); @@ -984,8 +987,6 @@ public class TestAssignmentManager { @Test public void testSSHTimesOutOpeningRegionTransition() throws KeeperException, IOException, ServiceException { - // We need a mocked catalog tracker. - CatalogTracker ct = Mockito.mock(CatalogTracker.class); // Create an AM. AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(this.server, this.serverManager); @@ -1001,7 +1002,7 @@ public class TestAssignmentManager { try { am.assignInvoked = false; - processServerShutdownHandler(ct, am, false); + processServerShutdownHandler(am.getCatalogTracker(), am, false); assertTrue(am.assignInvoked); } finally { am.getRegionStates().regionsInTransition.remove(REGIONINFO.getEncodedName()); @@ -1137,6 +1138,9 @@ public class TestAssignmentManager { HConnection connection = HConnectionTestingUtility. getMockedConnectionAndDecorate(HTU.getConfiguration(), null, ri, SERVERNAME_B, REGIONINFO); + // need a table stub so that the above ScanResponse mocking has something to hook onto + Mockito.when(connection.getTable(TableName.META_TABLE_NAME)) + .thenReturn(Mockito.mock(HTableInterface.class)); // Make it so we can get the connection from our mocked catalogtracker Mockito.when(ct.getConnection()).thenReturn(connection); // Create and startup an executor. Used by AM handling zk callbacks. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java index 8f0ae98..bf8eb05 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java @@ -53,8 +53,8 @@ import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.catalog.MetaMockingUtil; import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; +import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.io.Reference; @@ -133,7 +133,6 @@ public class TestCatalogJanitor { ServerName.valueOf("example.org,12345,6789"), HRegionInfo.FIRST_META_REGIONINFO); // Set hbase.rootdir into test dir. - FileSystem fs = FileSystem.get(this.c); Path rootdir = FSUtils.getRootDir(this.c); FSUtils.setRootDir(this.c, rootdir); this.ct = Mockito.mock(CatalogTracker.class); @@ -141,6 +140,9 @@ public class TestCatalogJanitor { Mockito.mock(AdminProtos.AdminService.BlockingInterface.class); Mockito.when(this.ct.getConnection()).thenReturn(this.connection); Mockito.when(ct.waitForMetaServerConnection(Mockito.anyLong())).thenReturn(hri); + // Mock a meta table that will accomidate delete() and close() calls. + HTable mockMeta = Mockito.mock(HTable.class); + Mockito.when(this.connection.getTable(TableName.META_TABLE_NAME)).thenReturn(mockMeta); } @Override @@ -184,7 +186,11 @@ public class TestCatalogJanitor { this.ct.stop(); } if (this.connection != null) { - HConnectionManager.deleteConnection(this.connection.getConfiguration()); + try { + this.connection.close(); + } catch (IOException e) { + // TODO: should be more careful here? + } } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java index 50678da..c1fcfdf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java @@ -199,7 +199,7 @@ public class TestMasterNoCluster { @Override CatalogTracker createCatalogTracker(ZooKeeperWatcher zk, - Configuration conf, Abortable abortable) + HConnection conn, Abortable abortable) throws IOException { // Insert a mock for the connection used by the CatalogTracker. Any // regionserver should do. Use TESTUTIL.getConfiguration rather than @@ -208,7 +208,7 @@ public class TestMasterNoCluster { HConnection connection = HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(), rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO); - return new CatalogTracker(zk, conf, connection, abortable); + return new CatalogTracker(zk, connection, abortable); } @Override @@ -289,7 +289,7 @@ public class TestMasterNoCluster { @Override CatalogTracker createCatalogTracker(ZooKeeperWatcher zk, - Configuration conf, Abortable abortable) + HConnection conn, Abortable abortable) throws IOException { // Insert a mock for the connection used by the CatalogTracker. Use // TESTUTIL.getConfiguration rather than the conf from the master; the @@ -298,7 +298,7 @@ public class TestMasterNoCluster { HConnection connection = HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(), rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO); - return new CatalogTracker(zk, conf, connection, abortable); + return new CatalogTracker(zk, connection, abortable); } @Override @@ -385,7 +385,7 @@ public class TestMasterNoCluster { @Override CatalogTracker createCatalogTracker(ZooKeeperWatcher zk, - Configuration conf, Abortable abortable) + HConnection conn, Abortable abortable) throws IOException { // Insert a mock for the connection used by the CatalogTracker. Any // regionserver should do. Use TESTUTIL.getConfiguration rather than @@ -394,7 +394,7 @@ public class TestMasterNoCluster { HConnection connection = HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(), rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO); - return new CatalogTracker(zk, conf, connection, abortable); + return new CatalogTracker(zk, connection, abortable); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java index dbc85ff..92978a8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java @@ -70,7 +70,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; - @Category(MediumTests.class) public class TestRegionPlacement { final static Log LOG = LogFactory.getLog(TestRegionPlacement.class); @@ -462,8 +461,6 @@ public class TestRegionPlacement { /** * Verify the meta has updated to the latest assignment plan - * @param plan - * @throws IOException */ private void verifyMETAUpdated(FavoredNodesPlan expectedPlan) throws IOException { @@ -527,8 +524,6 @@ public class TestRegionPlacement { /** * Verify the number of user regions is assigned to the primary * region server based on the plan is expected - * @param expectedNum. - * @throws IOException */ private void verifyRegionOnPrimaryRS(int expectedNum) throws IOException { @@ -655,7 +650,7 @@ public class TestRegionPlacement { @Override public void close() throws IOException {} }; - MetaScanner.metaScan(TEST_UTIL.getConfiguration(), visitor); + MetaScanner.metaScan(TEST_UTIL.getConnection(), visitor); LOG.info("There are " + regionOnPrimaryNum.intValue() + " out of " + totalRegionNum.intValue() + " regions running on the primary" + " region servers" ); @@ -664,10 +659,6 @@ public class TestRegionPlacement { /** * Create a table with specified table name and region number. - * @param tablename - * @param regionNum - * @return - * @throws IOException */ private static void createTable(byte[] tableName, int regionNum) throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java index 2cda4e9..8b2c58a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java @@ -27,6 +27,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.util.Bytes; @@ -99,7 +101,7 @@ public class TestRestartCluster { } List allRegions = - MetaScanner.listAllRegions(UTIL.getConfiguration(), true); + MetaScanner.listAllRegions(UTIL.getConnection(), true); assertEquals(4, allRegions.size()); LOG.info("\n\nShutting down cluster"); @@ -111,10 +113,12 @@ public class TestRestartCluster { LOG.info("\n\nStarting cluster the second time"); UTIL.restartHBaseCluster(3); - // Need to use a new 'Configuration' so we make a new HConnection. - // Otherwise we're reusing an HConnection that has gone stale because - // the shutdown of the cluster also called shut of the connection. - allRegions = MetaScanner.listAllRegions(new Configuration(UTIL.getConfiguration()), true); + // Need to use a new HConnection. Otherwise we're reusing an HConnection + // that has gone stale because the shutdown of the cluster also called + // shut of the connection. + HConnection conn = HConnectionManager.createConnection( + new Configuration(UTIL.getConfiguration())); + allRegions = MetaScanner.listAllRegions(conn, true); assertEquals(4, allRegions.size()); LOG.info("\n\nWaiting for tables to be available"); for(byte [] TABLE: TABLES) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java index 74ec2cb..b60c73f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java @@ -286,7 +286,7 @@ public class TestSnapshotFromMaster { UTIL.deleteTable(TABLE_NAME); HTableDescriptor htd = new HTableDescriptor(TABLE_NAME); htd.setCompactionEnabled(false); - UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration()); + UTIL.createTable(htd, new byte[][] { TEST_FAM }); // load the table (creates 4 hfiles) UTIL.loadTable(new HTable(UTIL.getConfiguration(), TABLE_NAME), TEST_FAM); UTIL.flush(TABLE_NAME); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index 438ed1e..b1872fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -99,8 +99,7 @@ public class TestEndToEndSplitTransaction { byte []firstRow = Bytes.toBytes("aaa"); byte []splitRow = Bytes.toBytes("lll"); byte []lastRow = Bytes.toBytes("zzz"); - HConnection con = HConnectionManager - .getConnection(TEST_UTIL.getConfiguration()); + HConnection con = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); // this will also cache the region byte[] regionName = con.locateRegion(tableName, splitRow).getRegionInfo() .getRegionName(); @@ -143,6 +142,8 @@ public class TestEndToEndSplitTransaction { regions.getSecond()); assertTrue(test(con, tableName, firstRow, server)); assertTrue(test(con, tableName, lastRow, server)); + + con.close(); } /** @@ -231,8 +232,8 @@ public class TestEndToEndSplitTransaction { try { Random random = new Random(); for (int i=0; i< 5; i++) { - NavigableMap regions = MetaScanner.allTableRegions(conf, null, - tableName, false); + NavigableMap regions = + MetaScanner.allTableRegions(TEST_UTIL.getConnection(), tableName, false); if (regions.size() == 0) { continue; } @@ -304,12 +305,12 @@ public class TestEndToEndSplitTransaction { void verifyRegionsUsingMetaScanner() throws Exception { //MetaScanner.allTableRegions() - NavigableMap regions = MetaScanner.allTableRegions(conf, null, - tableName, false); + NavigableMap regions = + MetaScanner.allTableRegions(TEST_UTIL.getConnection(), tableName, false); verifyTableRegions(regions.keySet()); //MetaScanner.listAllRegions() - List regionList = MetaScanner.listAllRegions(conf, false); + List regionList = MetaScanner.listAllRegions(TEST_UTIL.getConnection(), false); verifyTableRegions(Sets.newTreeSet(regionList)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java index e6845f7..5ae4c8d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java @@ -165,7 +165,7 @@ public class TestScannersWithLabels { public VisibilityLabelsResponse run() throws Exception { String[] labels = { SECRET, CONFIDENTIAL, PRIVATE, PUBLIC, TOPSECRET }; try { - VisibilityClient.addLabels(conf, labels); + VisibilityClient.addLabels(TEST_UTIL.getConnection(), labels); } catch (Throwable t) { throw new IOException(t); } @@ -177,7 +177,8 @@ public class TestScannersWithLabels { private static void setAuths() throws Exception { String[] labels = { SECRET, CONFIDENTIAL, PRIVATE, PUBLIC, TOPSECRET }; try { - VisibilityClient.setAuths(conf, labels, User.getCurrent().getShortName()); + VisibilityClient.setAuths(TEST_UTIL.getConnection(), labels, + User.getCurrent().getShortName()); } catch (Throwable t) { throw new IOException(t); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java index 6f468ca..ffdbc96 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java @@ -372,7 +372,7 @@ public class TestVisibilityLabels { public VisibilityLabelsResponse run() throws Exception { String[] labels = { SECRET, CONFIDENTIAL, PRIVATE, "ABC", "XYZ" }; try { - VisibilityClient.addLabels(conf, labels); + VisibilityClient.addLabels(TEST_UTIL.getConnection(), labels); } catch (Throwable t) { throw new IOException(t); } @@ -450,7 +450,7 @@ public class TestVisibilityLabels { String[] labels = { "L1", SECRET, "L2", "invalid~", "L3" }; VisibilityLabelsResponse response = null; try { - response = VisibilityClient.addLabels(conf, labels); + response = VisibilityClient.addLabels(TEST_UTIL.getConnection(), labels); } catch (Throwable e) { fail("Should not have thrown exception"); } @@ -476,7 +476,7 @@ public class TestVisibilityLabels { public Void run() throws Exception { String[] auths = { SECRET, CONFIDENTIAL }; try { - VisibilityClient.setAuths(conf, auths, user); + VisibilityClient.setAuths(TEST_UTIL.getConnection(), auths, user); } catch (Throwable e) { } return null; @@ -512,7 +512,7 @@ public class TestVisibilityLabels { public Void run() throws Exception { GetAuthsResponse authsResponse = null; try { - authsResponse = VisibilityClient.getAuths(conf, user); + authsResponse = VisibilityClient.getAuths(TEST_UTIL.getConnection(), user); } catch (Throwable e) { fail("Should not have failed"); } @@ -534,9 +534,9 @@ public class TestVisibilityLabels { String[] auths1 = { SECRET, CONFIDENTIAL }; GetAuthsResponse authsResponse = null; try { - VisibilityClient.setAuths(conf, auths1, user); + VisibilityClient.setAuths(TEST_UTIL.getConnection(), auths1, user); try { - authsResponse = VisibilityClient.getAuths(conf, user); + authsResponse = VisibilityClient.getAuths(TEST_UTIL.getConnection(), user); } catch (Throwable e) { fail("Should not have failed"); } @@ -562,7 +562,7 @@ public class TestVisibilityLabels { String[] auths = { SECRET, CONFIDENTIAL, PRIVATE }; String user = "testUser"; try { - VisibilityClient.setAuths(conf, auths, user); + VisibilityClient.setAuths(TEST_UTIL.getConnection(), auths, user); } catch (Throwable e) { fail("Should not have failed"); } @@ -571,7 +571,7 @@ public class TestVisibilityLabels { auths = new String[] { SECRET, PUBLIC, CONFIDENTIAL }; VisibilityLabelsResponse response = null; try { - response = VisibilityClient.clearAuths(conf, auths, user); + response = VisibilityClient.clearAuths(TEST_UTIL.getConnection(), auths, user); } catch (Throwable e) { fail("Should not have failed"); } @@ -604,7 +604,7 @@ public class TestVisibilityLabels { GetAuthsResponse authsResponse = null; try { - authsResponse = VisibilityClient.getAuths(conf, user); + authsResponse = VisibilityClient.getAuths(TEST_UTIL.getConnection(), user); } catch (Throwable e) { fail("Should not have failed"); } @@ -747,7 +747,7 @@ public class TestVisibilityLabels { public VisibilityLabelsResponse run() throws Exception { String[] labels = { SECRET, TOPSECRET, CONFIDENTIAL, PUBLIC, PRIVATE }; try { - VisibilityClient.addLabels(conf, labels); + VisibilityClient.addLabels(TEST_UTIL.getConnection(), labels); } catch (Throwable t) { throw new IOException(t); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOpWithDifferentUsersNoACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOpWithDifferentUsersNoACL.java index 3a12b00..f40b4be 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOpWithDifferentUsersNoACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOpWithDifferentUsersNoACL.java @@ -86,7 +86,8 @@ public class TestVisibilityLabelsOpWithDifferentUsersNoACL { new PrivilegedExceptionAction() { public VisibilityLabelsResponse run() throws Exception { try { - return VisibilityClient.setAuths(conf, new String[] { CONFIDENTIAL, PRIVATE }, "user1"); + return VisibilityClient.setAuths(TEST_UTIL.getConnection(), + new String[] { CONFIDENTIAL, PRIVATE }, "user1"); } catch (Throwable e) { } return null; @@ -100,7 +101,8 @@ public class TestVisibilityLabelsOpWithDifferentUsersNoACL { action = new PrivilegedExceptionAction() { public VisibilityLabelsResponse run() throws Exception { try { - return VisibilityClient.setAuths(conf, new String[] { CONFIDENTIAL, PRIVATE }, "user3"); + return VisibilityClient.setAuths(TEST_UTIL.getConnection(), + new String[] { CONFIDENTIAL, PRIVATE }, "user3"); } catch (Throwable e) { } return null; @@ -116,7 +118,7 @@ public class TestVisibilityLabelsOpWithDifferentUsersNoACL { new PrivilegedExceptionAction() { public GetAuthsResponse run() throws Exception { try { - return VisibilityClient.getAuths(conf, "user1"); + return VisibilityClient.getAuths(TEST_UTIL.getConnection(), "user1"); } catch (Throwable e) { } return null; @@ -139,7 +141,8 @@ public class TestVisibilityLabelsOpWithDifferentUsersNoACL { new PrivilegedExceptionAction() { public VisibilityLabelsResponse run() throws Exception { try { - return VisibilityClient.clearAuths(conf, new String[] { CONFIDENTIAL, PRIVATE }, "user1"); + return VisibilityClient.clearAuths(TEST_UTIL.getConnection(), + new String[] { CONFIDENTIAL, PRIVATE }, "user1"); } catch (Throwable e) { } return null; @@ -163,7 +166,7 @@ public class TestVisibilityLabelsOpWithDifferentUsersNoACL { public VisibilityLabelsResponse run() throws Exception { String[] labels = { SECRET, CONFIDENTIAL, PRIVATE }; try { - VisibilityClient.addLabels(conf, labels); + VisibilityClient.addLabels(TEST_UTIL.getConnection(), labels); } catch (Throwable t) { throw new IOException(t); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java index 15b3136..b4a3ebb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -67,6 +68,7 @@ public class TestVisibilityLabelsWithACL { private final static byte[] qual = Bytes.toBytes("qual"); private final static byte[] value = Bytes.toBytes("value"); private static Configuration conf; + private static HConnection conn; @Rule public final TestName TEST_NAME = new TestName(); @@ -84,6 +86,7 @@ public class TestVisibilityLabelsWithACL { conf.set("hbase.coprocessor.region.classes", AccessController.class.getName() + "," + VisibilityController.class.getName()); TEST_UTIL.startMiniCluster(2); + conn = TEST_UTIL.getConnection(); TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME.getName(), 50000); // Wait for the labels table to become available @@ -112,7 +115,7 @@ public class TestVisibilityLabelsWithACL { public void testScanForUserWithFewerLabelAuthsThanLabelsInScanAuthorizations() throws Throwable { String[] auths = { SECRET }; String user = "user2"; - VisibilityClient.setAuths(conf, auths, user); + VisibilityClient.setAuths(conn, auths, user); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final HTable table = createTableAndWriteDataWithLabels(tableName, SECRET + "&" + CONFIDENTIAL + "&!" + PRIVATE, SECRET + "&!" + PRIVATE); @@ -143,7 +146,7 @@ public class TestVisibilityLabelsWithACL { public void testScanForSuperUserWithFewerLabelAuths() throws Throwable { String[] auths = { SECRET }; String user = "admin"; - VisibilityClient.setAuths(conf, auths, user); + VisibilityClient.setAuths(conn, auths, user); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final HTable table = createTableAndWriteDataWithLabels(tableName, SECRET + "&" + CONFIDENTIAL + "&!" + PRIVATE, SECRET + "&!" + PRIVATE); @@ -169,7 +172,7 @@ public class TestVisibilityLabelsWithACL { public void testGetForSuperUserWithFewerLabelAuths() throws Throwable { String[] auths = { SECRET }; String user = "admin"; - VisibilityClient.setAuths(conf, auths, user); + VisibilityClient.setAuths(conn, auths, user); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final HTable table = createTableAndWriteDataWithLabels(tableName, SECRET + "&" + CONFIDENTIAL + "&!" + PRIVATE, SECRET + "&!" + PRIVATE); @@ -194,8 +197,8 @@ public class TestVisibilityLabelsWithACL { public void testVisibilityLabelsForUserWithNoAuths() throws Throwable { String user = "admin"; String[] auths = { SECRET }; - VisibilityClient.clearAuths(conf, auths, user); // Removing all auths if any. - VisibilityClient.setAuths(conf, auths, "user1"); + VisibilityClient.clearAuths(conn, auths, user); // Removing all auths if any. + VisibilityClient.setAuths(conn, auths, "user1"); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); final HTable table = createTableAndWriteDataWithLabels(tableName, SECRET); SecureTestUtil.grantOnTable(TEST_UTIL, NORMAL_USER1.getShortName(), tableName, @@ -225,7 +228,7 @@ public class TestVisibilityLabelsWithACL { new PrivilegedExceptionAction() { public VisibilityLabelsResponse run() throws Exception { try { - return VisibilityClient.addLabels(conf, new String[] { "l1", "l2" }); + return VisibilityClient.addLabels(conn, new String[] { "l1", "l2" }); } catch (Throwable e) { } return null; @@ -240,7 +243,7 @@ public class TestVisibilityLabelsWithACL { action = new PrivilegedExceptionAction() { public VisibilityLabelsResponse run() throws Exception { try { - return VisibilityClient.setAuths(conf, new String[] { CONFIDENTIAL, PRIVATE }, "user1"); + return VisibilityClient.setAuths(conn, new String[] { CONFIDENTIAL, PRIVATE }, "user1"); } catch (Throwable e) { } return null; @@ -255,7 +258,7 @@ public class TestVisibilityLabelsWithACL { action = new PrivilegedExceptionAction() { public VisibilityLabelsResponse run() throws Exception { try { - return VisibilityClient.setAuths(conf, new String[] { CONFIDENTIAL, PRIVATE }, "user1"); + return VisibilityClient.setAuths(conn, new String[] { CONFIDENTIAL, PRIVATE }, "user1"); } catch (Throwable e) { } return null; @@ -268,7 +271,7 @@ public class TestVisibilityLabelsWithACL { action = new PrivilegedExceptionAction() { public VisibilityLabelsResponse run() throws Exception { try { - return VisibilityClient.clearAuths(conf, new String[] { CONFIDENTIAL, PRIVATE }, "user1"); + return VisibilityClient.clearAuths(conn, new String[] { CONFIDENTIAL, PRIVATE }, "user1"); } catch (Throwable e) { } return null; @@ -280,16 +283,16 @@ public class TestVisibilityLabelsWithACL { assertEquals("org.apache.hadoop.hbase.security.AccessDeniedException", response.getResult(1) .getException().getName()); - response = VisibilityClient.clearAuths(conf, new String[] { CONFIDENTIAL, PRIVATE }, "user1"); + response = VisibilityClient.clearAuths(conn, new String[] { CONFIDENTIAL, PRIVATE }, "user1"); assertTrue(response.getResult(0).getException().getValue().isEmpty()); assertTrue(response.getResult(1).getException().getValue().isEmpty()); - VisibilityClient.setAuths(conf, new String[] { CONFIDENTIAL, PRIVATE }, "user3"); + VisibilityClient.setAuths(conn, new String[] { CONFIDENTIAL, PRIVATE }, "user3"); PrivilegedExceptionAction action1 = new PrivilegedExceptionAction() { public GetAuthsResponse run() throws Exception { try { - return VisibilityClient.getAuths(conf, "user3"); + return VisibilityClient.getAuths(conn, "user3"); } catch (Throwable e) { } return null; @@ -333,7 +336,7 @@ public class TestVisibilityLabelsWithACL { private static void addLabels() throws IOException { String[] labels = { SECRET, CONFIDENTIAL, PRIVATE }; try { - VisibilityClient.addLabels(conf, labels); + VisibilityClient.addLabels(conn, labels); } catch (Throwable t) { throw new IOException(t); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java index 629944b..73888ff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java @@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.PerformanceEvaluation; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.crypto.Cipher; @@ -478,12 +480,16 @@ public class LoadTestTool extends AbstractHBaseTool { AccessControlProtos.Permission.Action.CREATE, AccessControlProtos.Permission.Action.READ, AccessControlProtos.Permission.Action.WRITE }; + HConnection conn = null; try { - AccessControlClient.grant(conf, tableName, userOwner.getShortName(), COLUMN_FAMILY, + conn = HConnectionManager.createConnection(conf); + AccessControlClient.grant(conn, tableName, userOwner.getShortName(), COLUMN_FAMILY, null, actions); } catch (Throwable e) { LOG.fatal("Error in granting permission for the user " + userOwner.getShortName(), e); return EXIT_FAILURE; + } finally { + if (conn != null) conn.close(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java index 8cddae8..911757e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java @@ -117,6 +117,7 @@ public class TestHBaseFsck { private final static int REGION_ONLINE_TIMEOUT = 800; private static RegionStates regionStates; private static ExecutorService executorService; + private static HConnection conn; // for the instance, reset every test run private HTable tbl; @@ -139,6 +140,7 @@ public class TestHBaseFsck { AssignmentManager assignmentManager = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager(); regionStates = assignmentManager.getRegionStates(); + conn = TEST_UTIL.getConnection(); } @AfterClass @@ -1908,7 +1910,7 @@ public class TestHBaseFsck { // Mess it up by removing the RegionInfo for one region. final List deletes = new LinkedList(); HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName()); - MetaScanner.metaScan(conf, new MetaScanner.MetaScannerVisitor() { + MetaScanner.metaScan(conn, new MetaScanner.MetaScannerVisitor() { @Override public boolean processRow(Result rowResult) throws IOException { @@ -2134,7 +2136,7 @@ public class TestHBaseFsck { private void deleteMetaRegion(Configuration conf, boolean unassign, boolean hdfs, boolean regionInfoOnly) throws IOException, InterruptedException { - HConnection connection = HConnectionManager.getConnection(conf); + HConnection connection = HConnectionManager.createConnection(conf); HRegionLocation metaLocation = connection.locateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW); ServerName hsa = ServerName.valueOf(metaLocation.getHostnamePort(), 0L); @@ -2165,6 +2167,8 @@ public class TestHBaseFsck { LOG.info("Deleted " + p + " sucessfully? " + success); HBaseFsck.debugLsr(conf, p); } + + connection.close(); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java index c2e7ef1..e059de6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java @@ -111,7 +111,7 @@ public class TestMergeTable { LOG.info("Starting mini hbase cluster"); UTIL.startMiniHBaseCluster(1, 1); Configuration c = new Configuration(UTIL.getConfiguration()); - CatalogTracker ct = new CatalogTracker(c); + CatalogTracker ct = new CatalogTracker(UTIL.getConnection()); ct.start(); List originalTableRegions = MetaReader.getTableRegions(ct, desc.getTableName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java index 407be0d..e2a1616 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java @@ -269,7 +269,6 @@ public class TestRegionSplitter { * Creates a pre-split table with expectedBounds.size()+1 regions, then * verifies that the region boundaries are the same as the expected * region boundaries in expectedBounds. - * @throws Various junit assertions */ private void preSplitTableAndVerify(List expectedBounds, String splitClass, String tableName) throws Exception { @@ -277,8 +276,8 @@ public class TestRegionSplitter { final Configuration conf = UTIL.getConfiguration(); conf.setInt("split.count", numRegions); SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass); - RegionSplitter.createPresplitTable(tableName, splitAlgo, - new String[] {CF_NAME}, conf); + RegionSplitter.createPresplitTable(UTIL.getConnection(), tableName, splitAlgo, + new String[] {CF_NAME}); verifyBounds(expectedBounds, tableName); } @@ -289,7 +288,7 @@ public class TestRegionSplitter { // Set this larger than the number of splits so RegionSplitter won't block conf.setInt("split.outstanding", 5); SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass); - RegionSplitter.rollingSplit(tableName, splitAlgo, conf); + RegionSplitter.rollingSplit(UTIL.getConnection(), tableName, splitAlgo); verifyBounds(expectedBounds, tableName); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java index 4a5f84d..c25c65c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.catalog.MetaEditor; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -110,7 +109,6 @@ public class OfflineMetaRebuildTestCore { @After public void tearDownAfter() throws Exception { TEST_UTIL.shutdownMiniCluster(); - HConnectionManager.deleteConnection(conf); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java index 1a85bc5..d73b635 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java @@ -58,7 +58,6 @@ public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore { // shutdown the minicluster TEST_UTIL.shutdownMiniHBaseCluster(); TEST_UTIL.shutdownMiniZKCluster(); - HConnectionManager.deleteConnection(conf); // rebuild meta table from scratch HBaseFsck fsck = new HBaseFsck(conf); diff --git a/hbase-shell/pom.xml b/hbase-shell/pom.xml index f9a5179..103aa32 100644 --- a/hbase-shell/pom.xml +++ b/hbase-shell/pom.xml @@ -131,7 +131,7 @@ listener - org.apache.hadoop.hbase.ServerResourceCheckerJUnitListener + org.apache.hadoop.hbase.ResourceCheckerJUnitListener diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java index e469f6a..12c730d 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java @@ -37,10 +37,10 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTableInterface; -import org.apache.hadoop.hbase.client.HTablePool; import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.thrift.ThriftMetrics; import org.apache.hadoop.hbase.thrift2.generated.*; import org.apache.thrift.TException; @@ -52,17 +52,17 @@ import org.apache.thrift.TException; @InterfaceAudience.Private public class ThriftHBaseServiceHandler implements THBaseService.Iface { - // TODO: Size of pool configuraple - private final HTablePool htablePool; private static final Log LOG = LogFactory.getLog(ThriftHBaseServiceHandler.class); + private final HConnection conn; + // nextScannerId and scannerMap are used to manage scanner state // TODO: Cleanup thread for Scanners, Scanner id wrap private final AtomicInteger nextScannerId = new AtomicInteger(0); private final Map scannerMap = new ConcurrentHashMap(); - public static THBaseService.Iface newInstance(Configuration conf, ThriftMetrics metrics) { + public static THBaseService.Iface newInstance(Configuration conf, ThriftMetrics metrics) throws IOException { THBaseService.Iface handler = new ThriftHBaseServiceHandler(conf); return (THBaseService.Iface) Proxy.newProxyInstance(handler.getClass().getClassLoader(), new Class[] { THBaseService.Iface.class }, new THBaseServiceMetricsProxy(handler, metrics)); @@ -98,18 +98,19 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { return System.nanoTime(); } - ThriftHBaseServiceHandler(Configuration conf) { - int maxPoolSize = conf.getInt("hbase.thrift.htablepool.size.max", 1000); - htablePool = new HTablePool(conf, maxPoolSize); + ThriftHBaseServiceHandler(Configuration conf) throws IOException { + if (conf.getInt("hbase.thrift.htablepool.size.max", -1) != -1) + LOG.warn("hbase.thrift.htablepool.size.max is no longer supported. See HBASE-9117."); + conn = HConnectionManager.createConnection(conf); } - private HTableInterface getTable(ByteBuffer tableName) { - return htablePool.getTable(byteBufferToByteArray(tableName)); + private HTableInterface getTable(ByteBuffer tableName) throws IOException { + return conn.getTable(byteBufferToByteArray(tableName)); } private void closeTable(HTableInterface table) throws TIOError { try { - table.close(); + if (table != null) table.close(); } catch (IOException e) { throw getTIOError(e); } @@ -152,8 +153,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public boolean exists(ByteBuffer table, TGet get) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); return htable.exists(getFromThrift(get)); } catch (IOException e) { throw getTIOError(e); @@ -164,8 +166,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public TResult get(ByteBuffer table, TGet get) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); return resultFromHBase(htable.get(getFromThrift(get))); } catch (IOException e) { throw getTIOError(e); @@ -176,8 +179,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public List getMultiple(ByteBuffer table, List gets) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); return resultsFromHBase(htable.get(getsFromThrift(gets))); } catch (IOException e) { throw getTIOError(e); @@ -188,8 +192,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public void put(ByteBuffer table, TPut put) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); htable.put(putFromThrift(put)); } catch (IOException e) { throw getTIOError(e); @@ -201,8 +206,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public boolean checkAndPut(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TPut put) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); return htable.checkAndPut(byteBufferToByteArray(row), byteBufferToByteArray(family), byteBufferToByteArray(qualifier), (value == null) ? null : byteBufferToByteArray(value), putFromThrift(put)); @@ -215,8 +221,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public void putMultiple(ByteBuffer table, List puts) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); htable.put(putsFromThrift(puts)); } catch (IOException e) { throw getTIOError(e); @@ -227,8 +234,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public void deleteSingle(ByteBuffer table, TDelete deleteSingle) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); htable.delete(deleteFromThrift(deleteSingle)); } catch (IOException e) { throw getTIOError(e); @@ -240,8 +248,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public List deleteMultiple(ByteBuffer table, List deletes) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); htable.delete(deletesFromThrift(deletes)); } catch (IOException e) { throw getTIOError(e); @@ -254,9 +263,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public boolean checkAndDelete(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TDelete deleteSingle) throws TIOError, TException { - HTableInterface htable = getTable(table); - + HTableInterface htable = null; try { + htable = getTable(table); if (value == null) { return htable.checkAndDelete(byteBufferToByteArray(row), byteBufferToByteArray(family), byteBufferToByteArray(qualifier), null, deleteFromThrift(deleteSingle)); @@ -274,8 +283,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public TResult increment(ByteBuffer table, TIncrement increment) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); return resultFromHBase(htable.increment(incrementFromThrift(increment))); } catch (IOException e) { throw getTIOError(e); @@ -286,8 +296,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public TResult append(ByteBuffer table, TAppend append) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); return resultFromHBase(htable.append(appendFromThrift(append))); } catch (IOException e) { throw getTIOError(e); @@ -298,9 +309,10 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public int openScanner(ByteBuffer table, TScan scan) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; ResultScanner resultScanner = null; try { + htable = getTable(table); resultScanner = htable.getScanner(scanFromThrift(scan)); } catch (IOException e) { throw getTIOError(e); @@ -330,10 +342,11 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public List getScannerResults(ByteBuffer table, TScan scan, int numRows) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; List results = null; ResultScanner scanner = null; try { + htable = getTable(table); scanner = htable.getScanner(scanFromThrift(scan)); results = resultsFromHBase(scanner.next(numRows)); } catch (IOException e) { @@ -364,8 +377,9 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public void mutateRow(ByteBuffer table, TRowMutations rowMutations) throws TIOError, TException { - HTableInterface htable = getTable(table); + HTableInterface htable = null; try { + htable = getTable(table); htable.mutateRow(rowMutationsFromThrift(rowMutations)); } catch (IOException e) { throw getTIOError(e); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java index c701119..ffe7df2 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java @@ -18,8 +18,6 @@ */ package org.apache.hadoop.hbase.thrift2; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompatibilityFactory; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -45,7 +43,6 @@ import org.apache.hadoop.hbase.thrift2.generated.TDelete; import org.apache.hadoop.hbase.thrift2.generated.TDeleteType; import org.apache.hadoop.hbase.thrift2.generated.TGet; import org.apache.hadoop.hbase.thrift2.generated.THBaseService; -import org.apache.hadoop.hbase.thrift2.generated.TIOError; import org.apache.hadoop.hbase.thrift2.generated.TIllegalArgument; import org.apache.hadoop.hbase.thrift2.generated.TIncrement; import org.apache.hadoop.hbase.thrift2.generated.TPut; @@ -62,6 +59,7 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; @@ -85,7 +83,6 @@ import static java.nio.ByteBuffer.wrap; @Category(MediumTests.class) public class TestThriftHBaseServiceHandler { - public static final Log LOG = LogFactory.getLog(TestThriftHBaseServiceHandler.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); // Static names for tables, columns, rows, and values @@ -149,12 +146,12 @@ public class TestThriftHBaseServiceHandler { } - private ThriftHBaseServiceHandler createHandler() { + private ThriftHBaseServiceHandler createHandler() throws IOException { return new ThriftHBaseServiceHandler(UTIL.getConfiguration()); } @Test - public void testExists() throws TIOError, TException { + public void testExists() throws IOException, TException { ThriftHBaseServiceHandler handler = createHandler(); byte[] rowName = "testExists".getBytes(); ByteBuffer table = wrap(tableAname); @@ -669,7 +666,7 @@ public class TestThriftHBaseServiceHandler { // get scanner int scanId = handler.openScanner(table, scan); - List results = null; + List results; for (int i = 0; i < 10; i++) { // get batch for single row (10x10 is what we expect) results = handler.getScannerRows(scanId, 1); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java index 4bc633d..0340fc6 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java @@ -150,7 +150,7 @@ private static void createLabels() throws IOException, InterruptedException { public VisibilityLabelsResponse run() throws Exception { String[] labels = { SECRET, CONFIDENTIAL, PRIVATE, PUBLIC, TOPSECRET }; try { - VisibilityClient.addLabels(conf, labels); + VisibilityClient.addLabels(UTIL.getConnection(), labels); } catch (Throwable t) { throw new IOException(t); } @@ -163,7 +163,7 @@ private static void createLabels() throws IOException, InterruptedException { private static void setAuths() throws IOException { String[] labels = { SECRET, CONFIDENTIAL, PRIVATE, PUBLIC, TOPSECRET }; try { - VisibilityClient.setAuths(conf, labels, User.getCurrent().getShortName()); + VisibilityClient.setAuths(UTIL.getConnection(), labels, User.getCurrent().getShortName()); } catch (Throwable t) { throw new IOException(t); } @@ -179,7 +179,7 @@ public void setup() throws Exception { } -private ThriftHBaseServiceHandler createHandler() { +private ThriftHBaseServiceHandler createHandler() throws IOException { return new ThriftHBaseServiceHandler(UTIL.getConfiguration()); }