diff --git bin/region_mover.rb bin/region_mover.rb
index 565b0d5..cd0f173 100644
--- bin/region_mover.rb
+++ bin/region_mover.rb
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin
import org.apache.hadoop.hbase.client.Get
import org.apache.hadoop.hbase.client.Scan
import org.apache.hadoop.hbase.client.HTable
-import org.apache.hadoop.hbase.client.HConnectionManager
+import org.apache.hadoop.hbase.client.ConnectionFactory
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
import org.apache.hadoop.hbase.filter.FilterList;
@@ -243,7 +243,7 @@ end
# Now get list of regions on targetServer
def getRegions(config, servername)
- connection = HConnectionManager::getConnection(config);
+ connection = ConnectionFactory::createConnection(config);
return ProtobufUtil::getOnlineRegions(connection.getAdmin(ServerName.valueOf(servername)));
end
diff --git bin/region_status.rb bin/region_status.rb
index a016afd..52af49e 100644
--- bin/region_status.rb
+++ bin/region_status.rb
@@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.HRegionInfo
import org.apache.hadoop.hbase.MetaTableAccessor
import org.apache.hadoop.hbase.HTableDescriptor
-import org.apache.hadoop.hbase.client.HConnectionManager
+import org.apache.hadoop.hbase.client.ConnectionFactory
# disable debug logging on this script for clarity
log_level = org.apache.log4j.Level::ERROR
@@ -138,7 +138,7 @@ while true
if $tablename.nil?
server_count = admin.getClusterStatus().getRegionsCount()
else
- connection = HConnectionManager::getConnection(config);
+ connection = ConnectionFactory::createConnection(config);
server_count = MetaTableAccessor::allTableRegions(connection, $TableName).size()
end
print "Region Status: #{server_count} / #{meta_count}\n"
diff --git conf/log4j.properties conf/log4j.properties
index 40f47ba..7b0acc0 100644
--- conf/log4j.properties
+++ conf/log4j.properties
@@ -82,7 +82,7 @@ log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
#log4j.logger.org.apache.hadoop.dfs=DEBUG
# Set this class to log INFO only otherwise its OTT
# Enable this to get detailed connection error/retry logging.
-# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE
+# log4j.logger.org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation=TRACE
# Uncomment this line to enable tracing on _every_ RPC call (this can be a lot of output)
@@ -90,4 +90,4 @@ log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
# Uncomment the below if you want to remove logging of client region caching'
# and scan of hbase:meta messages
-# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=INFO
+# log4j.logger.org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation=INFO
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java
index 37f1a33..362439a 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java
@@ -20,7 +20,7 @@ import java.util.concurrent.ExecutorService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
/**
* Coprocessor environment state.
@@ -50,14 +50,14 @@ public interface CoprocessorEnvironment {
* @return an interface for accessing the given table
* @throws IOException
*/
- HTableInterface getTable(TableName tableName) throws IOException;
+ Table getTable(TableName tableName) throws IOException;
/**
* @return an interface for accessing the given table using the passed executor to run batch
* operations
* @throws IOException
*/
- HTableInterface getTable(TableName tableName, ExecutorService service) throws IOException;
+ Table getTable(TableName tableName, ExecutorService service) throws IOException;
/**
* @return the classloader for the loaded coprocessor instance
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 15d325d..4df58a2 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.NeedUnmanagedConnectionException;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@@ -263,7 +263,7 @@ public class MetaTableAccessor {
// There should still be a way to use this method with an unmanaged connection.
if (connection instanceof ClusterConnection) {
if (((ClusterConnection) connection).isManaged()) {
- return new HTable(TableName.META_TABLE_NAME, connection);
+ throw new NeedUnmanagedConnectionException();
}
}
return connection.getTable(TableName.META_TABLE_NAME);
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
index 89378dd..0b4b29a 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
@@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.security.UserProvider;
* Similarly, {@link Connection} also returns {@link Admin} and {@link RegionLocator}
* implementations.
*
- * This class replaces {@link HConnectionManager}, which is now deprecated.
* @see Connection
* @since 0.99.0
*/
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
index c32d74d..b9cf4c0 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
@@ -286,11 +286,11 @@ final class ConnectionManager {
synchronized (CONNECTION_INSTANCES) {
HConnectionImplementation connection = CONNECTION_INSTANCES.get(connectionKey);
if (connection == null) {
- connection = (HConnectionImplementation)createConnection(conf, true);
+ connection = (HConnectionImplementation) ConnectionFactory.createConnection(conf);
CONNECTION_INSTANCES.put(connectionKey, connection);
} else if (connection.isClosed()) {
ConnectionManager.deleteConnection(connectionKey, true);
- connection = (HConnectionImplementation)createConnection(conf, true);
+ connection = (HConnectionImplementation) ConnectionFactory.createConnection(conf);
CONNECTION_INSTANCES.put(connectionKey, connection);
}
connection.incCount();
@@ -335,7 +335,7 @@ final class ConnectionManager {
* This is the recommended way to create HConnections.
* {@code
* ExecutorService pool = ...;
- * HConnection connection = HConnectionManager.createConnection(conf, pool);
+ * HConnection connection = ConnectionManager.createConnection(conf, pool);
* HTableInterface table = connection.getTable("mytable");
* table.get(...);
* ...
@@ -361,7 +361,7 @@ final class ConnectionManager {
* This is the recommended way to create HConnections.
* {@code
* ExecutorService pool = ...;
- * HConnection connection = HConnectionManager.createConnection(conf, pool);
+ * HConnection connection = ConnectionManager.createConnection(conf, pool);
* HTableInterface table = connection.getTable("mytable");
* table.get(...);
* ...
@@ -386,7 +386,7 @@ final class ConnectionManager {
* This is the recommended way to create HConnections.
* {@code
* ExecutorService pool = ...;
- * HConnection connection = HConnectionManager.createConnection(conf, pool);
+ * HConnection connection = ConnectionManager.createConnection(conf, pool);
* HTableInterface table = connection.getTable("mytable");
* table.get(...);
* ...
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
index f185cb2..e4f05b0 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
/**
* A cluster connection. Knows how to find the master, locate regions out on the cluster,
* keeps a cache of locations and then knows how to re-calibrate after they move. You need one
- * of these to talk to your HBase cluster. {@link HConnectionManager} manages instances of this
+ * of these to talk to your HBase cluster. {@link ConnectionFactory} manages instances of this
* class. See it for how to get one of these.
*
*
This is NOT a connection to a particular server but to ALL servers in the cluster. Individual
@@ -49,11 +49,12 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
* HConnection instances can be shared. Sharing
* is usually what you want because rather than each HConnection instance
* having to do its own discovery of regions out on the cluster, instead, all
- * clients get to share the one cache of locations. {@link HConnectionManager} does the
+ * clients get to share the one cache of locations. {@link ConnectionManager} does the
* sharing for you if you go by it getting connections. Sharing makes cleanup of
- * HConnections awkward. See {@link HConnectionManager} for cleanup discussion.
+ * HConnections awkward. See {@link ConnectionFactory} for cleanup discussion.
*
- * @see HConnectionManager
+ * @see ConnectionManager
+ * @see ConnectionFactory
* @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
*/
@InterfaceAudience.Public
@@ -79,7 +80,7 @@ public interface HConnection extends Connection {
* This is a lightweight operation, pooling or caching of the returned HTableInterface
* is neither required nor desired.
* Note that the HConnection needs to be unmanaged
- * (created with {@link HConnectionManager#createConnection(Configuration)}).
+ * (created with {@link ConnectionFactory#createConnection(Configuration)}).
* @param tableName
* @return an HTable to use for interactions with this table
*/
@@ -92,7 +93,7 @@ public interface HConnection extends Connection {
* This is a lightweight operation, pooling or caching of the returned HTableInterface
* is neither required nor desired.
* Note that the HConnection needs to be unmanaged
- * (created with {@link HConnectionManager#createConnection(Configuration)}).
+ * (created with {@link ConnectionFactory#createConnection(Configuration)}).
* @param tableName
* @return an HTable to use for interactions with this table
*/
@@ -105,7 +106,7 @@ public interface HConnection extends Connection {
* This is a lightweight operation, pooling or caching of the returned HTableInterface
* is neither required nor desired.
* Note that the HConnection needs to be unmanaged
- * (created with {@link HConnectionManager#createConnection(Configuration)}).
+ * (created with {@link ConnectionFactory#createConnection(Configuration)}).
* @param tableName
* @return an HTable to use for interactions with this table
*/
@@ -119,7 +120,7 @@ public interface HConnection extends Connection {
* This is a lightweight operation, pooling or caching of the returned HTableInterface
* is neither required nor desired.
* Note that the HConnection needs to be unmanaged
- * (created with {@link HConnectionManager#createConnection(Configuration)}).
+ * (created with {@link ConnectionFactory#createConnection(Configuration)}).
* @param tableName
* @param pool The thread pool to use for batch operations, null to use a default pool.
* @return an HTable to use for interactions with this table
@@ -133,7 +134,7 @@ public interface HConnection extends Connection {
* This is a lightweight operation, pooling or caching of the returned HTableInterface
* is neither required nor desired.
* Note that the HConnection needs to be unmanaged
- * (created with {@link HConnectionManager#createConnection(Configuration)}).
+ * (created with {@link ConnectionFactory#createConnection(Configuration)}).
* @param tableName
* @param pool The thread pool to use for batch operations, null to use a default pool.
* @return an HTable to use for interactions with this table
@@ -147,7 +148,7 @@ public interface HConnection extends Connection {
* This is a lightweight operation, pooling or caching of the returned HTableInterface
* is neither required nor desired.
* Note that the HConnection needs to be unmanaged
- * (created with {@link HConnectionManager#createConnection(Configuration)}).
+ * (created with {@link ConnectionFactory#createConnection(Configuration)}).
* @param tableName
* @param pool The thread pool to use for batch operations, null to use a default pool.
* @return an HTable to use for interactions with this table
@@ -163,7 +164,7 @@ public interface HConnection extends Connection {
* required nor desired.
*
* RegionLocator needs to be unmanaged
- * (created with {@link HConnectionManager#createConnection(Configuration)}).
+ * (created with {@link ConnectionFactory#createConnection(Configuration)}).
*
* @param tableName Name of the table who's region is to be examined
* @return A RegionLocator instance
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
deleted file mode 100644
index edd071b..0000000
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
+++ /dev/null
@@ -1,324 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.client;
-
-import java.io.IOException;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.commons.logging.Log;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.security.User;
-
-/**
- * A non-instantiable class that manages creation of {@link HConnection}s.
- *
The simplest way to use this class is by using {@link #createConnection(Configuration)}.
- * This creates a new {@link HConnection} to the cluster that is managed by the caller.
- * From this {@link HConnection} {@link HTableInterface} implementations are retrieved
- * with {@link HConnection#getTable(byte[])}. Example:
- *
- * HConnection connection = HConnectionManager.createConnection(config);
- * HTableInterface table = connection.getTable(TableName.valueOf("table1"));
- * try {
- * // Use the table as needed, for a single operation and a single thread
- * } finally {
- * table.close();
- * connection.close();
- * }
- *
- * This class has a static Map of {@link HConnection} instances keyed by
- * {@link HConnectionKey}; A {@link HConnectionKey} is identified by a set of
- * {@link Configuration} properties. Invocations of {@link #getConnection(Configuration)}
- * that pass the same {@link Configuration} instance will return the same
- * {@link HConnection} instance ONLY WHEN the set of properties are the same
- * (i.e. if you change properties in your {@link Configuration} instance, such as RPC timeout,
- * the codec used, HBase will create a new {@link HConnection} instance. For more details 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.
- * 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
- * set property "hbase.client.instance.id" to a unique value for your {@link Configuration}
- * instance, like the following:
- *
- * {@code
- * conf.set("hbase.client.instance.id", "12345");
- * HConnection connection = HConnectionManager.getConnection(conf);
- * // Use the connection to your hearts' delight and then when done...
- * conf.set("hbase.client.instance.id", "12345");
- * HConnectionManager.deleteConnection(conf, 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.
- * @deprecated Please use ConnectionFactory instead
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-@Deprecated
-public final class HConnectionManager extends ConnectionFactory {
-
- /** @deprecated connection caching is going away */
- @Deprecated
- public static final String RETRIES_BY_SERVER_KEY =
- ConnectionManager.RETRIES_BY_SERVER_KEY;
-
- /** @deprecated connection caching is going away */
- @Deprecated
- public static final int MAX_CACHED_CONNECTION_INSTANCES =
- ConnectionManager.MAX_CACHED_CONNECTION_INSTANCES;
-
- /*
- * Non-instantiable.
- */
- private HConnectionManager() {
- super();
- }
-
- /**
- * 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
- * @deprecated connection caching is going away
- */
- @Deprecated
- 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.
- *
- * This is the recommended way to create HConnections.
- *
- * HConnection connection = HConnectionManager.createConnection(conf);
- * HTableInterface table = connection.getTable("mytable");
- * try {
- * table.get(...);
- * ...
- * } finally {
- * table.close();
- * connection.close();
- * }
- *
- *
- * @param conf configuration
- * @return HConnection object for conf
- * @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
- */
- @Deprecated
- public static HConnection createConnection(Configuration conf) throws IOException {
- return ConnectionManager.createConnectionInternal(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.
- *
- * ExecutorService pool = ...;
- * HConnection connection = HConnectionManager.createConnection(conf, pool);
- * HTableInterface table = connection.getTable("mytable");
- * table.get(...);
- * ...
- * table.close();
- * connection.close();
- *
- * @param conf configuration
- * @param pool the thread pool to use for batch operation in HTables used via this HConnection
- * @return HConnection object for conf
- * @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
- */
- @Deprecated
- public static HConnection createConnection(Configuration conf, ExecutorService pool)
- throws IOException {
- return ConnectionManager.createConnection(conf, pool);
- }
-
- /**
- * 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.
- *
- * ExecutorService pool = ...;
- * HConnection connection = HConnectionManager.createConnection(conf, pool);
- * HTableInterface table = connection.getTable("mytable");
- * table.get(...);
- * ...
- * table.close();
- * connection.close();
- *
- * @param conf configuration
- * @param user the user the connection is for
- * @return HConnection object for conf
- * @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
- */
- @Deprecated
- public static HConnection createConnection(Configuration conf, User user)
- throws IOException {
- return ConnectionManager.createConnection(conf, 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.
- *
- * ExecutorService pool = ...;
- * HConnection connection = HConnectionManager.createConnection(conf, pool);
- * HTableInterface table = connection.getTable("mytable");
- * table.get(...);
- * ...
- * table.close();
- * connection.close();
- *
- * @param conf configuration
- * @param pool the thread pool to use for batch operation in HTables used via this HConnection
- * @param user the user the connection is for
- * @return HConnection object for conf
- * @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
- */
- @Deprecated
- public static HConnection createConnection(Configuration conf, ExecutorService pool, User user)
- throws IOException {
- return ConnectionManager.createConnection(conf, pool, user);
- }
-
- /**
- * @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
- */
- @Deprecated
- static HConnection createConnection(final Configuration conf, final boolean managed)
- throws IOException {
- return ConnectionManager.createConnection(conf, managed);
- }
-
- /**
- * @deprecated in favor of {@link Connection} and {@link ConnectionFactory}
- */
- @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 connection caching is going away.
- */
- @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 connection caching is going away.
- */
- @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 connection caching is going away.
- */
- @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
- * duration of the invocation.
- *
- * @param the return type of the connect method
- * @param connectable the {@link HConnectable} instance
- * @return the value returned by the connect method
- * @throws IOException
- * @deprecated Internal method, do not use thru HConnectionManager.
- */
- @InterfaceAudience.Private
- @Deprecated
- public static T execute(HConnectable connectable) throws IOException {
- return ConnectionManager.execute(connectable);
- }
-
- /**
- * Set the number of retries to use serverside when trying to communicate
- * with another server over {@link HConnection}. Used updating catalog
- * tables, etc. Call this method before we create any Connections.
- * @param c The Configuration instance to set the retries into.
- * @param log Used to log what we set in here.
- * @deprecated Internal method, do not use.
- */
- @InterfaceAudience.Private
- @Deprecated
- public static void setServerSideHConnectionRetries(
- final Configuration c, final String sn, final Log log) {
- ConnectionUtils.setServerSideHConnectionRetriesConfig(c, sn, log);
- }
-}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 4ad9eac..c77e2ae 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -128,78 +128,6 @@ public class HTable implements HTableInterface {
private RpcRetryingCallerFactory rpcCallerFactory;
private RpcControllerFactory rpcControllerFactory;
- /**
- * Creates an object to access a HBase table.
- * @param conf Configuration object to use.
- * @param tableName Name of the table.
- * @throws IOException if a remote or network exception occurs
- * @deprecated Constructing HTable objects manually has been deprecated. Please use
- * {@link Connection} to instantiate a {@link Table} instead.
- */
- @Deprecated
- public HTable(Configuration conf, final String tableName)
- throws IOException {
- this(conf, TableName.valueOf(tableName));
- }
-
- /**
- * Creates an object to access a HBase table.
- * @param conf Configuration object to use.
- * @param tableName Name of the table.
- * @throws IOException if a remote or network exception occurs
- * @deprecated Constructing HTable objects manually has been deprecated. Please use
- * {@link Connection} to instantiate a {@link Table} instead.
- */
- @Deprecated
- public HTable(Configuration conf, final byte[] tableName)
- throws IOException {
- this(conf, TableName.valueOf(tableName));
- }
-
- /**
- * Creates an object to access a HBase table.
- * @param conf Configuration object to use.
- * @param tableName table name pojo
- * @throws IOException if a remote or network exception occurs
- * @deprecated Constructing HTable objects manually has been deprecated. Please use
- * {@link Connection} to instantiate a {@link Table} instead.
- */
- @Deprecated
- public HTable(Configuration conf, final TableName tableName)
- throws IOException {
- this.tableName = tableName;
- this.cleanupPoolOnClose = true;
- this.cleanupConnectionOnClose = true;
- if (conf == null) {
- this.connection = null;
- return;
- }
- this.connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
- this.configuration = conf;
-
- this.pool = getDefaultExecutor(conf);
- this.finishSetup();
- }
-
- /**
- * Creates an object to access a HBase table.
- * @param tableName Name of the table.
- * @param connection HConnection to be used.
- * @throws IOException if a remote or network exception occurs
- * @deprecated Do not use.
- */
- @Deprecated
- public HTable(TableName tableName, Connection connection) throws IOException {
- this.tableName = tableName;
- this.cleanupPoolOnClose = true;
- this.cleanupConnectionOnClose = false;
- this.connection = (ClusterConnection)connection;
- this.configuration = connection.getConfiguration();
-
- this.pool = getDefaultExecutor(this.configuration);
- this.finishSetup();
- }
-
// Marked Private @since 1.0
@InterfaceAudience.Private
public static ThreadPoolExecutor getDefaultExecutor(Configuration conf) {
@@ -221,68 +149,6 @@ public class HTable implements HTableInterface {
/**
* Creates an object to access a HBase table.
- * @param conf Configuration object to use.
- * @param tableName Name of the table.
- * @param pool ExecutorService to be used.
- * @throws IOException if a remote or network exception occurs
- * @deprecated Constructing HTable objects manually has been deprecated. Please use
- * {@link Connection} to instantiate a {@link Table} instead.
- */
- @Deprecated
- public HTable(Configuration conf, final byte[] tableName, final ExecutorService pool)
- throws IOException {
- this(conf, TableName.valueOf(tableName), pool);
- }
-
- /**
- * Creates an object to access a HBase table.
- * @param conf Configuration object to use.
- * @param tableName Name of the table.
- * @param pool ExecutorService to be used.
- * @throws IOException if a remote or network exception occurs
- * @deprecated Constructing HTable objects manually has been deprecated. Please use
- * {@link Connection} to instantiate a {@link Table} instead.
- */
- @Deprecated
- public HTable(Configuration conf, final TableName tableName, final ExecutorService pool)
- throws IOException {
- this.connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
- this.configuration = conf;
- this.pool = pool;
- if (pool == null) {
- this.pool = getDefaultExecutor(conf);
- this.cleanupPoolOnClose = true;
- } else {
- this.cleanupPoolOnClose = false;
- }
- this.tableName = tableName;
- this.cleanupConnectionOnClose = true;
- this.finishSetup();
- }
-
- /**
- * Creates an object to access a HBase table.
- * @param tableName Name of the table.
- * @param connection HConnection to be used.
- * @param pool ExecutorService to be used.
- * @throws IOException if a remote or network exception occurs.
- * @deprecated Do not use, internal ctor.
- */
- @Deprecated
- public HTable(final byte[] tableName, final Connection connection,
- final ExecutorService pool) throws IOException {
- this(TableName.valueOf(tableName), connection, pool);
- }
-
- /** @deprecated Do not use, internal ctor. */
- @Deprecated
- public HTable(TableName tableName, final Connection connection,
- final ExecutorService pool) throws IOException {
- this(tableName, (ClusterConnection)connection, null, null, null, pool);
- }
-
- /**
- * Creates an object to access a HBase table.
* Used by HBase internally. DO NOT USE. See {@link ConnectionFactory} class comment for how to
* get a {@link Table} instance (use {@link Table} instead of {@link HTable}).
* @param tableName Name of the table.
@@ -291,7 +157,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 TableConfiguration tableConfig,
final RpcRetryingCallerFactory rpcCallerFactory,
final RpcControllerFactory rpcControllerFactory,
@@ -453,7 +319,7 @@ public class HTable implements HTableInterface {
@Deprecated
public static boolean isTableEnabled(Configuration conf,
final TableName tableName) throws IOException {
- return HConnectionManager.execute(new HConnectable(conf) {
+ return ConnectionManager.execute(new HConnectable(conf) {
@Override
public Boolean connect(HConnection connection) throws IOException {
return connection.isTableEnabled(tableName);
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java
deleted file mode 100644
index 6970333..0000000
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java
+++ /dev/null
@@ -1,51 +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.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * Factory for creating HTable instances.
- *
- * @deprecated as of 0.98.1. See {@link HConnectionManager#createConnection(Configuration)}.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-@Deprecated
-public class HTableFactory implements HTableInterfaceFactory {
- @Override
- public HTableInterface createHTableInterface(Configuration config,
- byte[] tableName) {
- try {
- return new HTable(config, TableName.valueOf(tableName));
- } catch (IOException ioe) {
- throw new RuntimeException(ioe);
- }
- }
-
- @Override
- public void releaseHTableInterface(HTableInterface table) throws IOException {
- table.close();
- }
-}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java
deleted file mode 100644
index b6349c2..0000000
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java
+++ /dev/null
@@ -1,54 +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.conf.Configuration;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-
-/**
- * Defines methods to create new HTableInterface.
- *
- * @since 0.21.0
- * @deprecated in favor of {@link ConnectionFactory} and {@link Connection}.
- */
-@Deprecated
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public interface HTableInterfaceFactory {
-
- /**
- * Creates a new HTableInterface.
- *
- * @param config HBaseConfiguration instance.
- * @param tableName name of the HBase table.
- * @return HTableInterface instance.
- */
- HTableInterface createHTableInterface(Configuration config, byte[] tableName);
-
-
- /**
- * Release the HTable resource represented by the table.
- * @param table
- */
- void releaseHTableInterface(final HTableInterface table) throws IOException;
-}
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java
index b44803b..6110f0d 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* Container for Actions (i.e. Get, Delete, or Put), which are grouped by
- * regionName. Intended to be used with HConnectionManager.processBatch()
+ * regionName. Intended to be used with ConnectionManager.processBatch()
*/
@InterfaceAudience.Private
public final class MultiAction {
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
index 4a57adf..a9c903e 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
@@ -129,7 +129,7 @@ public class ReversedScannerCallable extends ScannerCallable {
}
// check how often we retry.
- // HConnectionManager will call instantiateServer with reload==true
+ // ConnectionManager will call instantiateServer with reload==true
// if and only if for retries.
if (reload && this.scanMetrics != null) {
this.scanMetrics.countOfRPCRetries.incrementAndGet();
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
index 6d5bb9e..226782c 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
@@ -153,7 +153,7 @@ public class ScannerCallable extends RegionServerCallable {
}
// check how often we retry.
- // HConnectionManager will call instantiateServer with reload==true
+ // ConnectionManager will call instantiateServer with reload==true
// if and only if for retries.
if (reload && this.scanMetrics != null) {
this.scanMetrics.countOfRPCRetries.incrementAndGet();
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosingException.java hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosingException.java
index 49134f1..43a4ee4 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosingException.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ConnectionClosingException.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
* Thrown when the client believes that we are trying to communicate to has
* been repeatedly unresponsive for a while.
*
-* On receiving such an exception. The HConnectionManager will skip all
+* On receiving such an exception. The ConnectionManager will skip all
* retries and fast fail the operation.
*/
@InterfaceAudience.Public
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/PreemptiveFastFailException.java hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/PreemptiveFastFailException.java
index 51c960d..6ca1d88 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/PreemptiveFastFailException.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/PreemptiveFastFailException.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.ServerName;
* Thrown when the client believes that we are trying to communicate to has
* been repeatedly unresponsive for a while.
*
- * On receiving such an exception. The HConnectionManager will skip all
+ * On receiving such an exception. The ConnectionManager will skip all
* retries and fast fail the operation.
*/
@InterfaceAudience.Public
diff --git hbase-client/src/test/resources/log4j.properties hbase-client/src/test/resources/log4j.properties
index 6ee91ef..69171f7 100644
--- hbase-client/src/test/resources/log4j.properties
+++ hbase-client/src/test/resources/log4j.properties
@@ -63,4 +63,4 @@ log4j.logger.org.apache.hadoop.hbase=DEBUG
log4j.org.apache.hadoop.metrics2.impl.MetricsSystemImpl=ERROR
log4j.org.apache.hadoop.metrics2.util.MBeans=ERROR
# Enable this to get detailed connection error/retry logging.
-# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE
+# log4j.logger.org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation=TRACE
diff --git hbase-common/src/test/resources/log4j.properties hbase-common/src/test/resources/log4j.properties
index 6ee91ef..69171f7 100644
--- hbase-common/src/test/resources/log4j.properties
+++ hbase-common/src/test/resources/log4j.properties
@@ -63,4 +63,4 @@ log4j.logger.org.apache.hadoop.hbase=DEBUG
log4j.org.apache.hadoop.metrics2.impl.MetricsSystemImpl=ERROR
log4j.org.apache.hadoop.metrics2.util.MBeans=ERROR
# Enable this to get detailed connection error/retry logging.
-# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE
+# log4j.logger.org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation=TRACE
diff --git hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
index 99ad36a..f378064 100644
--- hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
+++ hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
@@ -40,10 +40,10 @@ import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.BufferedMutatorParams;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
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.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@@ -448,7 +448,7 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
@Override
protected void handleFailure(Counters counters) throws IOException {
Configuration conf = job.getConfiguration();
- HConnection conn = HConnectionManager.getConnection(conf);
+ HConnection conn = (HConnection) ConnectionFactory.createConnection(conf);
TableName tableName = TableName.valueOf(COMMON_TABLE_NAME);
CounterGroup g = counters.getGroup("undef");
Iterator it = g.iterator();
diff --git hbase-rest/src/test/resources/log4j.properties hbase-rest/src/test/resources/log4j.properties
index 6ee91ef..69171f7 100644
--- hbase-rest/src/test/resources/log4j.properties
+++ hbase-rest/src/test/resources/log4j.properties
@@ -63,4 +63,4 @@ log4j.logger.org.apache.hadoop.hbase=DEBUG
log4j.org.apache.hadoop.metrics2.impl.MetricsSystemImpl=ERROR
log4j.org.apache.hadoop.metrics2.util.MBeans=ERROR
# Enable this to get detailed connection error/retry logging.
-# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE
+# log4j.logger.org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation=TRACE
diff --git hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index ceda843..b3445d4 100644
--- hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -42,7 +42,6 @@ org.apache.hadoop.hbase.NamespaceDescriptor;
org.apache.hadoop.hbase.ServerLoad;
org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.client.Admin;
-org.apache.hadoop.hbase.client.HConnectionManager;
org.apache.hadoop.hbase.HRegionInfo;
org.apache.hadoop.hbase.master.RegionState;
org.apache.hadoop.hbase.HTableDescriptor;
diff --git hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
index f063e74..49addc7 100644
--- hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
+++ hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
@@ -34,7 +34,6 @@ HMaster master;
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;
%import>
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
index eacba6f..17fc34f 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
@@ -173,7 +173,7 @@ public class LocalHBaseCluster {
throws IOException {
// Create each regionserver with its own Configuration instance so each has
// its HConnection instance rather than share (see HBASE_INSTANCES down in
- // the guts of HConnectionManager.
+ // the guts of ConnectionManager).
// Also, create separate CoordinatedStateManager instance per Server.
// This is special case when we have to have more than 1 CoordinatedStateManager
@@ -206,7 +206,7 @@ public class LocalHBaseCluster {
throws IOException {
// Create each master with its own Configuration instance so each has
// its HConnection instance rather than share (see HBASE_INSTANCES down in
- // the guts of HConnectionManager.
+ // the guts of ConnectionManager.
// Also, create separate CoordinatedStateManager instance per Server.
// This is special case when we have to have more than 1 CoordinatedStateManager
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
index c16b4c3..e3641c7 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
@@ -58,23 +58,23 @@ import com.google.protobuf.ServiceException;
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Stable
-public final class HTableWrapper implements HTableInterface {
+public final class HTableWrapper implements Table {
- private final HTableInterface table;
+ private final Table table;
private ClusterConnection connection;
- private final List openTables;
+ private final List openTables;
/**
* @param openTables External list of tables used for tracking wrappers.
* @throws IOException
*/
- public static HTableInterface createWrapper(List openTables,
+ public static Table createWrapper(List openTables,
TableName tableName, Environment env, ExecutorService pool) throws IOException {
return new HTableWrapper(openTables, tableName,
CoprocessorHConnection.getConnectionForEnvironment(env), pool);
}
- private HTableWrapper(List openTables, TableName tableName,
+ private HTableWrapper(List openTables, TableName tableName,
ClusterConnection connection, ExecutorService pool)
throws IOException {
this.table = connection.getTable(tableName, pool);
@@ -116,8 +116,7 @@ public final class HTableWrapper implements HTableInterface {
}
/**
- * @deprecated in 0.99 since setting clearBufferOnFail is deprecated. Use
- * {@link #setAutoFlushTo(boolean)}} instead.
+ * @deprecated in 0.99 since setting clearBufferOnFail is deprecated.
*/
@Deprecated
public Result getRowOrBefore(byte[] row, byte[] family)
@@ -215,14 +214,6 @@ public final class HTableWrapper implements HTableInterface {
return table.increment(increment);
}
- public void flushCommits() throws IOException {
- table.flushCommits();
- }
-
- public boolean isAutoFlush() {
- return table.isAutoFlush();
- }
-
public ResultScanner getScanner(Scan scan) throws IOException {
return table.getScanner(scan);
}
@@ -241,11 +232,6 @@ public final class HTableWrapper implements HTableInterface {
}
@Override
- public byte[] getTableName() {
- return table.getTableName();
- }
-
- @Override
public TableName getName() {
return table.getName();
}
@@ -317,30 +303,6 @@ public final class HTableWrapper implements HTableInterface {
}
@Override
- public void setAutoFlush(boolean autoFlush) {
- table.setAutoFlush(autoFlush);
- }
-
- @Override
- public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
- setAutoFlush(autoFlush);
- if (!autoFlush && !clearBufferOnFail) {
- // We don't support his combination. In HTable, the implementation is this:
- //
- // this.clearBufferOnFail = autoFlush || clearBufferOnFail
- //
- // So if autoFlush == false and clearBufferOnFail is false, that is not supported in
- // the new Table Interface so just throwing UnsupportedOperationException here.
- throw new UnsupportedOperationException("Can't do this via wrapper");
- }
- }
-
- @Override
- public void setAutoFlushTo(boolean autoFlush) {
- table.setAutoFlushTo(autoFlush);
- }
-
- @Override
public long getWriteBufferSize() {
return table.getWriteBufferSize();
}
@@ -351,13 +313,6 @@ public final class HTableWrapper implements HTableInterface {
}
@Override
- public long incrementColumnValue(byte[] row, byte[] family,
- byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
- return table.incrementColumnValue(row, family, qualifier, amount,
- writeToWAL? Durability.USE_DEFAULT: Durability.SKIP_WAL);
- }
-
- @Override
public Map batchCoprocessorService(
MethodDescriptor methodDescriptor, Message request, byte[] startKey, byte[] endKey,
R responsePrototype) throws ServiceException, Throwable {
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
index 237f617..b047d33 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
@@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.HTableWrapper;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet;
import org.apache.hadoop.hbase.util.VersionInfo;
@@ -377,8 +377,8 @@ public abstract class CoprocessorHost {
/** Current coprocessor state */
Coprocessor.State state = Coprocessor.State.UNINSTALLED;
/** Accounting for tables opened by the coprocessor */
- protected List openTables =
- Collections.synchronizedList(new ArrayList());
+ protected List openTables =
+ Collections.synchronizedList(new ArrayList());
private int seq;
private Configuration conf;
private ClassLoader classLoader;
@@ -438,7 +438,7 @@ public abstract class CoprocessorHost {
" because not active (state="+state.toString()+")");
}
// clean up any table references
- for (HTableInterface table: openTables) {
+ for (Table table: openTables) {
try {
((HTableWrapper)table).internalClose();
} catch (IOException e) {
@@ -493,7 +493,7 @@ public abstract class CoprocessorHost {
* @exception java.io.IOException Exception
*/
@Override
- public HTableInterface getTable(TableName tableName) throws IOException {
+ public Table getTable(TableName tableName) throws IOException {
return this.getTable(tableName, HTable.getDefaultExecutor(getConfiguration()));
}
@@ -504,7 +504,7 @@ public abstract class CoprocessorHost {
* @exception java.io.IOException Exception
*/
@Override
- public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException {
+ public Table getTable(TableName tableName, ExecutorService pool) throws IOException {
return HTableWrapper.createWrapper(openTables, tableName, this, pool);
}
}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
index dd58d5c..f8ccea3 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter;
@@ -83,7 +84,8 @@ public abstract class TableInputFormatBase
implements InputFormat {
private static final Log LOG = LogFactory.getLog(TableInputFormatBase.class);
private byte [][] inputColumns;
- private HTable table;
+ private Table table;
+ private RegionLocator regionLocator;
private Connection connection;
private TableRecordReader tableRecordReader;
private Filter rowFilter;
@@ -197,7 +199,7 @@ implements InputFormat {
throw new IOException(INITIALIZATION_ERROR, exception);
}
- byte [][] startKeys = this.table.getStartKeys();
+ byte [][] startKeys = this.regionLocator.getStartKeys();
if (startKeys == null || startKeys.length == 0) {
throw new IOException("Expecting at least one region");
}
@@ -212,7 +214,7 @@ implements InputFormat {
for (int i = 0; i < realNumSplits; i++) {
int lastPos = startPos + middle;
lastPos = startKeys.length % realNumSplits > i ? lastPos + 1 : lastPos;
- String regionLocation = table.getRegionLocation(startKeys[startPos]).
+ String regionLocation = regionLocator.getRegionLocation(startKeys[startPos]).
getHostname();
splits[i] = new TableSplit(this.table.getName(),
startKeys[startPos], ((i + 1) < realNumSplits) ? startKeys[lastPos]:
@@ -235,7 +237,8 @@ implements InputFormat {
LOG.warn("initializeTable called multiple times. Overwriting connection and table " +
"reference; TableInputFormatBase will not close these old references when done.");
}
- this.table = (HTable) connection.getTable(tableName);
+ this.table = connection.getTable(tableName);
+ this.regionLocator = connection.getRegionLocator(tableName);
this.connection = connection;
}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index b6d43de..da04565 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -30,9 +30,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HConnectable;
-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.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -129,22 +126,18 @@ public class VerifyReplication extends Configured implements Tool {
}
final TableSplit tableSplit = (TableSplit)(context.getInputSplit());
- HConnectionManager.execute(new HConnectable(conf) {
- @Override
- public Void connect(HConnection conn) throws IOException {
- String zkClusterKey = conf.get(NAME + ".peerQuorumAddress");
- Configuration peerConf = HBaseConfiguration.create(conf);
- ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey);
-
- TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName"));
- connection = ConnectionFactory.createConnection(peerConf);
- replicatedTable = connection.getTable(tableName);
- scan.setStartRow(value.getRow());
- scan.setStopRow(tableSplit.getEndRow());
- replicatedScanner = replicatedTable.getScanner(scan);
- return null;
- }
- });
+
+ String zkClusterKey = conf.get(NAME + ".peerQuorumAddress");
+ Configuration peerConf = HBaseConfiguration.create(conf);
+ ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey);
+
+ TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName"));
+ connection = ConnectionFactory.createConnection(peerConf);
+ replicatedTable = connection.getTable(tableName);
+ scan.setStartRow(value.getRow());
+ scan.setStopRow(tableSplit.getEndRow());
+ replicatedScanner = replicatedTable.getScanner(scan);
+
currentCompareRowInPeerTable = replicatedScanner.next();
}
while (true) {
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index 95c253d..1a53c24 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -30,8 +30,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -82,7 +82,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
// 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.createConnection(this.conf);
+ this.conn = (HConnection) ConnectionFactory.createConnection(this.conf);
this.sleepForRetries =
this.conf.getLong("replication.source.sleepforretries", 1000);
this.metrics = context.getMetrics();
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java
index 21714af..bbfed4c 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/ConnectionCache.java
@@ -27,11 +27,12 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
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.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.security.UserGroupInformation;
@@ -146,9 +147,9 @@ public class ConnectionCache {
/**
* Caller closes the table afterwards.
*/
- public HTableInterface getTable(String tableName) throws IOException {
+ public Table getTable(String tableName) throws IOException {
ConnectionInfo connInfo = getCurrentConnection();
- return connInfo.connection.getTable(tableName);
+ return connInfo.connection.getTable(TableName.valueOf(tableName));
}
/**
@@ -168,7 +169,7 @@ public class ConnectionCache {
ugi = UserGroupInformation.createProxyUser(userName, realUser);
}
User user = userProvider.create(ugi);
- HConnection conn = HConnectionManager.createConnection(conf, user);
+ Connection conn = ConnectionFactory.createConnection(conf, user);
connInfo = new ConnectionInfo(conn, userName);
connections.put(userName, connInfo);
}
@@ -180,14 +181,14 @@ public class ConnectionCache {
}
class ConnectionInfo {
- final HConnection connection;
+ final Connection connection;
final String userName;
volatile HBaseAdmin admin;
private long lastAccessTime;
private boolean closed;
- ConnectionInfo(HConnection conn, String user) {
+ ConnectionInfo(Connection conn, String user) {
lastAccessTime = EnvironmentEdgeManager.currentTime();
connection = conn;
closed = false;
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
index 78c7a06..8507df3 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
@@ -41,9 +41,7 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnectable;
import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
@@ -106,14 +104,14 @@ class HMerge {
final TableName tableName, final boolean testMasterRunning)
throws IOException {
boolean masterIsRunning = false;
+ HConnection hConnection = null;
if (testMasterRunning) {
- masterIsRunning = HConnectionManager
- .execute(new HConnectable(conf) {
- @Override
- public Boolean connect(HConnection connection) throws IOException {
- return connection.isMasterRunning();
- }
- });
+ try {
+ hConnection = (HConnection) ConnectionFactory.createConnection(conf);
+ masterIsRunning = hConnection.isMasterRunning();
+ } finally {
+ hConnection.close();
+ }
}
if (tableName.equals(TableName.META_TABLE_NAME)) {
if (masterIsRunning) {
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
index 81678aa..ba76c6d 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java
@@ -35,8 +35,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -64,7 +64,7 @@ public class MultiHConnection {
synchronized (this.hConnectionsLock) {
hConnections = new HConnection[noOfConnections];
for (int i = 0; i < noOfConnections; i++) {
- HConnection conn = HConnectionManager.createConnection(conf);
+ HConnection conn = (HConnection) ConnectionFactory.createConnection(conf);
hConnections[i] = conn;
}
}
diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index e3e6bc4..517b67e 100644
--- hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -70,11 +70,12 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.TableState;
@@ -2140,7 +2141,7 @@ public class WALSplitter {
synchronized (this.tableNameToHConnectionMap) {
hconn = this.tableNameToHConnectionMap.get(tableName);
if (hconn == null) {
- hconn = HConnectionManager.getConnection(conf);
+ hconn = (HConnection) ConnectionFactory.createConnection(conf);
this.tableNameToHConnectionMap.put(tableName, hconn);
}
}
diff --git hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp
index 831835e..50a7560 100644
--- hbase-server/src/main/resources/hbase-webapps/master/snapshot.jsp
+++ 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.Admin"
- 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"
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 26290d6..cf476bb 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1034,7 +1034,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
// Don't leave here till we've done a successful scan of the hbase:meta
- Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
+ Connection conn = ConnectionFactory.createConnection(this.conf);
+ Table t = conn.getTable(TableName.META_TABLE_NAME);
ResultScanner s = t.getScanner(new Scan());
while (s.next() != null) {
// do nothing
@@ -1042,6 +1043,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
LOG.info("HBase has been restarted");
s.close();
t.close();
+ conn.close();
}
/**
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
index 4439d4c..9d30764 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
@@ -38,8 +38,6 @@ import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -285,9 +283,9 @@ public class TestZooKeeper {
// make sure they aren't the same
ZooKeeperWatcher z1 =
- getZooKeeperWatcher(HConnectionManager.getConnection(localMeta.getConfiguration()));
+ getZooKeeperWatcher(ConnectionFactory.createConnection(localMeta.getConfiguration()));
ZooKeeperWatcher z2 =
- getZooKeeperWatcher(HConnectionManager.getConnection(otherConf));
+ getZooKeeperWatcher(ConnectionFactory.createConnection(otherConf));
assertFalse(z1 == z2);
assertFalse(z1.getQuorum().equals(z2.getQuorum()));
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index 998cdf0..39a4ae3 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -39,14 +39,14 @@ public class HConnectionTestingUtility {
/*
* Not part of {@link HBaseTestingUtility} because this class is not
* in same package as {@link HConnection}. Would have to reveal ugly
- * {@link HConnectionManager} innards to HBaseTestingUtility to give it access.
+ * {@link ConnectionManager} innards to HBaseTestingUtility to give it access.
*/
/**
* Get a Mocked {@link HConnection} that goes with the passed conf
* configuration instance. Minimally the mock will return
* conf when {@link ClusterConnection#getConfiguration()} is invoked.
* Be sure to shutdown the connection when done by calling
- * {@link HConnectionManager#deleteConnection(Configuration)} else it
+ * {@link ConnectionManager#deleteConnection(Configuration)} else it
* will stick around; this is probably not what you want.
* @param conf configuration
* @return HConnection object for conf
@@ -72,7 +72,7 @@ public class HConnectionTestingUtility {
* more of the popular {@link ClusterConnection} 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
+ * {@link ConnectionManager#deleteConnection(Configuration)} else it
* will stick around; this is probably not what you want.
*
* @param conf Configuration to use
@@ -92,7 +92,7 @@ public class HConnectionTestingUtility {
* {@link ClusterConnection#getAdmin(ServerName)} is called, returns the passed
* {@link ClientProtos.ClientService.BlockingInterface} instance when
* {@link ClusterConnection#getClient(ServerName)} is called (Be sure to call
- * {@link HConnectionManager#deleteConnection(Configuration)}
+ * {@link ConnectionManager#deleteConnection(Configuration)}
* when done with this mocked Connection.
* @throws IOException
*/
@@ -146,7 +146,7 @@ public class HConnectionTestingUtility {
* Get a Mockito spied-upon {@link ClusterConnection} that goes with the passed
* conf configuration instance.
* Be sure to shutdown the connection when done by calling
- * {@link HConnectionManager#deleteConnection(Configuration)} else it
+ * {@link ConnectionManager#deleteConnection(Configuration)} else it
* will stick around; this is probably not what you want.
* @param conf configuration
* @return HConnection object for conf
@@ -161,7 +161,7 @@ public class HConnectionTestingUtility {
HConnectionImplementation connection =
ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
if (connection == null) {
- connection = Mockito.spy(new HConnectionImplementation(conf, true));
+ connection = Mockito.spy(new HConnectionImplementation(conf, false));
ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
}
return connection;
@@ -175,7 +175,7 @@ public class HConnectionTestingUtility {
HConnectionImplementation connection =
ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
if (connection == null) {
- connection = Mockito.spy(new HConnectionImplementation(conf, true));
+ connection = Mockito.spy(new HConnectionImplementation(conf, false));
ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
}
return connection;
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index b3c631a..40fbf6e 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -307,7 +307,7 @@ public class TestFromClientSide {
// Then a ZooKeeperKeepAliveConnection
ConnectionManager.HConnectionImplementation connection1 =
(ConnectionManager.HConnectionImplementation)
- HConnectionManager.getConnection(newConfig);
+ ConnectionFactory.createConnection(newConfig);
ZooKeeperKeepAliveConnection z1 = connection1.getKeepAliveZooKeeperWatcher();
z1.getRecoverableZooKeeper().getZooKeeper().exists("/z1", false);
@@ -329,7 +329,7 @@ public class TestFromClientSide {
newConfig2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "6789");
ConnectionManager.HConnectionImplementation connection2 =
(ConnectionManager.HConnectionImplementation)
- HConnectionManager.getConnection(newConfig2);
+ ConnectionFactory.createConnection(newConfig2);
assertTrue("connections should be different ", connection1 != connection2);
@@ -355,7 +355,7 @@ public class TestFromClientSide {
z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false);
- HConnectionManager.deleteConnection(newConfig);
+ ConnectionManager.deleteConnection(newConfig);
try {
z2.getRecoverableZooKeeper().getZooKeeper().exists("/z2", false);
assertTrue("We should not have a valid connection for z2", false);
@@ -366,7 +366,7 @@ public class TestFromClientSide {
// We expect success here.
- HConnectionManager.deleteConnection(newConfig2);
+ ConnectionManager.deleteConnection(newConfig2);
try {
z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false);
assertTrue("We should not have a valid connection for z4", false);
@@ -4127,7 +4127,7 @@ public class TestFromClientSide {
*/
HTable createUnmangedHConnectionHTable(final TableName tableName) throws IOException {
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
- HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
+ HConnection conn = ConnectionManager.createConnection(TEST_UTIL.getConfiguration());
return (HTable)conn.getTable(tableName);
}
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
index 219496f..78be899 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
@@ -155,8 +155,8 @@ public class TestHCM {
new SynchronousQueue(),
Threads.newDaemonThreadFactory("test-hcm"));
- HConnection con1 = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
- HConnection con2 = HConnectionManager.createConnection(TEST_UTIL.getConfiguration(), otherPool);
+ HConnection con1 = ConnectionManager.createConnection(TEST_UTIL.getConfiguration());
+ HConnection con2 = ConnectionManager.createConnection(TEST_UTIL.getConfiguration(), otherPool);
// make sure the internally created ExecutorService is the one passed
assertTrue(otherPool == ((HConnectionImplementation)con2).getCurrentBatchPool());
@@ -537,7 +537,7 @@ public class TestHCM {
} finally {
syncBlockingFilter.set(true);
t.join();
- HConnectionManager.getConnection(c2).close();
+ ConnectionManager.getConnection(c2).close();
TEST_UTIL.getHBaseAdmin().setBalancerRunning(previousBalance, true);
}
@@ -580,11 +580,11 @@ public class TestHCM {
ConnectionManager.CONNECTION_INSTANCES.clear();
try {
- HConnection connection = HConnectionManager.getConnection(TEST_UTIL.getConfiguration());
+ HConnection connection = ConnectionManager.getConnection(TEST_UTIL.getConfiguration());
connection.abort("test abortingHConnectionRemovesItselfFromHCM", new Exception(
"test abortingHConnectionRemovesItselfFromHCM"));
Assert.assertNotSame(connection,
- HConnectionManager.getConnection(TEST_UTIL.getConfiguration()));
+ ConnectionManager.getConnection(TEST_UTIL.getConfiguration()));
} finally {
// Put original HConnections back
ConnectionManager.CONNECTION_INSTANCES.clear();
@@ -861,7 +861,7 @@ public class TestHCM {
configuration.set("some_key", String.valueOf(_randy.nextInt()));
LOG.info("The hash code of the current configuration is: "
+ configuration.hashCode());
- Connection currentConnection = HConnectionManager
+ Connection currentConnection = ConnectionManager
.getConnection(configuration);
if (previousConnection != null) {
assertTrue(
@@ -879,7 +879,7 @@ public class TestHCM {
/**
* Makes sure that there is no leaking of
- * {@link ConnectionManager.HConnectionImplementation} in the {@link HConnectionManager}
+ * {@link ConnectionManager.HConnectionImplementation} in the {@link ConnectionManager}
* class.
* @deprecated Tests deprecated functionality. Remove in 1.0.
*/
@@ -905,7 +905,7 @@ public class TestHCM {
LOG.info("The hash code of the current configuration is: "
+ configuration.hashCode());
HConnection currentConnection =
- HConnectionManager.getConnection(configuration);
+ ConnectionManager.getConnection(configuration);
if (previousConnection != null) {
assertTrue("Got the same connection even though its key changed!",
previousConnection != currentConnection);
@@ -917,7 +917,7 @@ public class TestHCM {
configuration.set("other_key", String.valueOf(_randy.nextInt()));
previousConnection = currentConnection;
- LOG.info("The current HConnectionManager#HBASE_INSTANCES cache size is: "
+ LOG.info("The current ConnectionManager#HBASE_INSTANCES cache size is: "
+ getHConnectionManagerCacheSize());
Thread.sleep(50);
connections.add(currentConnection);
@@ -925,7 +925,7 @@ public class TestHCM {
} finally {
for (Connection c: connections) {
// Clean up connections made so we don't interfere w/ subsequent tests.
- HConnectionManager.deleteConnection(c.getConfiguration());
+ ConnectionManager.deleteConnection(c.getConfiguration());
}
}
}
@@ -941,8 +941,8 @@ public class TestHCM {
// We create two connections with the same key.
Connection c2 = ConnectionFactory.createConnection(configuration);
- Connection c3 = HConnectionManager.getConnection(configuration);
- Connection c4 = HConnectionManager.getConnection(configuration);
+ Connection c3 = ConnectionFactory.createConnection(configuration);
+ Connection c4 = ConnectionFactory.createConnection(configuration);
assertTrue(c3 == c4);
c1.close();
@@ -952,11 +952,9 @@ public class TestHCM {
c3.close();
// still a reference left
- assertFalse(c3.isClosed());
- c3.close();
assertTrue(c3.isClosed());
- // c3 was removed from the cache
- Connection c5 = HConnectionManager.getConnection(configuration);
+
+ Connection c5 = ConnectionManager.getConnection(configuration);
assertTrue(c5 != c3);
assertFalse(c2.isClosed());
@@ -968,7 +966,7 @@ public class TestHCM {
/**
* Trivial test to verify that nobody messes with
- * {@link HConnectionManager#createConnection(Configuration)}
+ * {@link ConnectionFactory#createConnection(Configuration)}
*/
@Test
public void testCreateConnection() throws Exception {
@@ -979,7 +977,7 @@ public class TestHCM {
assertTrue(c1 != c2);
assertTrue(c1.getConfiguration() == c2.getConfiguration());
// make sure these were not cached
- Connection c3 = HConnectionManager.getConnection(configuration);
+ Connection c3 = ConnectionManager.getConnection(configuration);
assertTrue(c1 != c3);
assertTrue(c2 != c3);
}
@@ -1000,7 +998,7 @@ public class TestHCM {
TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_CLIENT_PORT));
// This should be enough to connect
- HConnection conn = HConnectionManager.getConnection(c);
+ HConnection conn = ConnectionManager.getConnection(c);
assertTrue( conn.isMasterRunning() );
conn.close();
}
@@ -1224,9 +1222,9 @@ public class TestHCM {
public void run() {
while (!Thread.interrupted()) {
try {
- HConnection conn = HConnectionManager.getConnection(config);
+ HConnection conn = ConnectionManager.getConnection(config);
LOG.info("Connection " + conn);
- HConnectionManager.deleteStaleConnection(conn);
+ ConnectionManager.deleteStaleConnection(conn);
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
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
index 317707a..8efbfbf 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@@ -86,7 +85,7 @@ public class TestHTableWrapper {
static class DummyRegionObserver extends BaseRegionObserver {
}
- private HTableInterface hTableInterface;
+ private Table hTableInterface;
private Table table;
@BeforeClass
@@ -144,10 +143,8 @@ public class TestHTableWrapper {
private void checkHTableInterfaceMethods() throws Exception {
checkConf();
checkNameAndDescriptor();
- checkAutoFlush();
checkBufferSize();
checkExists();
- checkGetRowOrBefore();
checkAppend();
checkPutsAndDeletes();
checkCheckAndPut();
@@ -159,7 +156,6 @@ public class TestHTableWrapper {
checkMutateRow();
checkResultScanner();
- hTableInterface.flushCommits();
hTableInterface.close();
}
@@ -174,15 +170,6 @@ public class TestHTableWrapper {
assertEquals(table.getTableDescriptor(), hTableInterface.getTableDescriptor());
}
- private void checkAutoFlush() {
- boolean initialAutoFlush = hTableInterface.isAutoFlush();
- hTableInterface.setAutoFlush(false);
- assertFalse(hTableInterface.isAutoFlush());
- hTableInterface.setAutoFlush(true);
- assertTrue(hTableInterface.isAutoFlush());
- hTableInterface.setAutoFlush(initialAutoFlush);
- }
-
private void checkBufferSize() throws IOException {
long initialWriteBufferSize = hTableInterface.getWriteBufferSize();
hTableInterface.setWriteBufferSize(12345L);
@@ -194,19 +181,12 @@ public class TestHTableWrapper {
boolean ex = hTableInterface.exists(new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1));
assertTrue(ex);
- Boolean[] exArray = hTableInterface.exists(Arrays.asList(new Get[] {
- new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1),
- new Get(ROW_B).addColumn(TEST_FAMILY, qualifierCol1),
- new Get(ROW_C).addColumn(TEST_FAMILY, qualifierCol1),
- new Get(Bytes.toBytes("does not exist")).addColumn(TEST_FAMILY, qualifierCol1), }));
- assertArrayEquals(new Boolean[] { Boolean.TRUE, Boolean.TRUE, Boolean.TRUE, Boolean.FALSE },
- exArray);
- }
-
- @SuppressWarnings("deprecation")
- private void checkGetRowOrBefore() throws IOException {
- Result rowOrBeforeResult = hTableInterface.getRowOrBefore(ROW_A, TEST_FAMILY);
- assertArrayEquals(ROW_A, rowOrBeforeResult.getRow());
+ boolean[] exArray = hTableInterface.existsAll(Arrays.asList(new Get[]{
+ new Get(ROW_A).addColumn(TEST_FAMILY, qualifierCol1),
+ new Get(ROW_B).addColumn(TEST_FAMILY, qualifierCol1),
+ new Get(ROW_C).addColumn(TEST_FAMILY, qualifierCol1),
+ new Get(Bytes.toBytes("does not exist")).addColumn(TEST_FAMILY, qualifierCol1),}));
+ assertTrue(Arrays.equals(new boolean[]{true, true, true, false}, exArray));
}
private void checkAppend() throws IOException {
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java
index d7dd8ec..3497cdf 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java
@@ -403,10 +403,10 @@ public class TestTableInputFormat {
@Override
public void configure(JobConf job) {
try {
- HTable exampleTable = new HTable(HBaseConfiguration.create(job),
- Bytes.toBytes("exampleDeprecatedTable"));
+ Connection connection = ConnectionFactory.createConnection(job);
+ Table exampleTable = connection.getTable(TableName.valueOf("exampleDeprecatedTable"));
// mandatory
- setHTable(exampleTable);
+ initializeTable(connection, exampleTable.getName());
byte[][] inputColumns = new byte [][] { Bytes.toBytes("columnA"),
Bytes.toBytes("columnB") };
// mandatory
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java
index 566a642..bc2d08f 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormat.java
@@ -412,10 +412,10 @@ public class TestTableInputFormat {
@Override
public void configure(JobConf job) {
try {
- HTable exampleTable = new HTable(HBaseConfiguration.create(job),
- Bytes.toBytes("exampleDeprecatedTable"));
+ Connection connection = ConnectionFactory.createConnection(job);
+ Table exampleTable = connection.getTable(TableName.valueOf(("exampleDeprecatedTable")));
// mandatory
- setHTable(exampleTable);
+ initializeTable(connection, exampleTable.getName());
byte[][] inputColumns = new byte [][] { Bytes.toBytes("columnA"),
Bytes.toBytes("columnB") };
// optional
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
index a7025c6..30a0e88 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
@@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
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.Put;
import org.apache.hadoop.hbase.client.Result;
@@ -107,8 +106,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 = (HConnection) ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
// this will also cache the region
byte[] regionName = con.locateRegion(tableName, splitRow).getRegionInfo()
.getRegionName();
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 06f9eb8..9d14be6 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -57,6 +57,8 @@ import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@@ -958,7 +960,8 @@ public class TestSplitTransactionOnCluster {
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
admin.createTable(desc);
- HTable hTable = new HTable(cluster.getConfiguration(), desc.getTableName());
+ Connection connection = ConnectionFactory.createConnection(cluster.getConfiguration());
+ HTable hTable = (HTable) connection.getTable(desc.getTableName());
for(int i = 1; i < 5; i++) {
Put p1 = new Put(("r"+i).getBytes());
p1.add(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
index a501af9..0ec410e 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
@@ -37,11 +37,11 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
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.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
@@ -350,7 +350,7 @@ public class TestMasterReplication {
int numClusters = utilities.length;
Table[] htables = new Table[numClusters];
for (int i = 0; i < numClusters; i++) {
- Table htable = new HTable(configurations[i], tableName);
+ Table htable = ConnectionFactory.createConnection(configurations[i]).getTable(tableName);
htable.setWriteBufferSize(1024);
htables[i] = htable;
}
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
index 6f46fd3..3c71a8c 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
@@ -195,13 +194,16 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
List cells = null;
if (labelsRegion == null) {
Table table = null;
+ Connection connection = null;
try {
- table = new HTable(conf, VisibilityConstants.LABELS_TABLE_NAME);
+ connection = ConnectionFactory.createConnection(conf);
+ table = connection.getTable(VisibilityConstants.LABELS_TABLE_NAME);
Result result = table.get(get);
cells = result.listCells();
} finally {
if (table != null) {
table.close();
+ connection.close();
}
}
} else {
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
index 5b04ab9..26f2db9 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java
@@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@@ -155,7 +155,7 @@ public abstract class MultiThreadedAction {
this.dataGenerator = dataGen;
this.tableName = tableName;
this.actionLetter = actionLetter;
- this.connection = HConnectionManager.createConnection(conf);
+ this.connection = (HConnection) ConnectionFactory.createConnection(conf);
}
public void start(long startKey, long endKey, int numThreads) throws IOException {
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
index 2b16dce..ba7ba9c 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
@@ -1219,7 +1219,7 @@ public class TestHBaseFsck {
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
admin.createTable(desc);
- tbl = new HTable(cluster.getConfiguration(), desc.getTableName());
+ tbl = (HTable) connection.getTable(desc.getTableName());
for (int i = 0; i < 5; i++) {
Put p1 = new Put(("r" + i).getBytes());
p1.add(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
index fc22292..d4f86e9 100644
--- hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
+++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.HBaseFsck;
diff --git hbase-server/src/test/resources/log4j.properties hbase-server/src/test/resources/log4j.properties
index 6ee91ef..69171f7 100644
--- hbase-server/src/test/resources/log4j.properties
+++ hbase-server/src/test/resources/log4j.properties
@@ -63,4 +63,4 @@ log4j.logger.org.apache.hadoop.hbase=DEBUG
log4j.org.apache.hadoop.metrics2.impl.MetricsSystemImpl=ERROR
log4j.org.apache.hadoop.metrics2.util.MBeans=ERROR
# Enable this to get detailed connection error/retry logging.
-# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE
+# log4j.logger.org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation=TRACE
diff --git hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/HTablePool.java hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/HTablePool.java
index e9c9e1f..7be3dcc 100644
--- hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/HTablePool.java
+++ hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/HTablePool.java
@@ -30,12 +30,10 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableFactory;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.HTableInterfaceFactory;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@@ -43,6 +41,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@@ -66,10 +65,10 @@ import com.google.protobuf.ServiceException;
* HTableInterface.close() rather than returning the tables to the pool
*
* Once you are done with it, close your instance of
- * {@link org.apache.hadoop.hbase.client.HTableInterface}
- * by calling {@link org.apache.hadoop.hbase.client.HTableInterface#close()} rather than returning
+ * {@link org.apache.hadoop.hbase.client.Table}
+ * by calling {@link org.apache.hadoop.hbase.client.Table#close()} rather than returning
* the tablesto the pool with (deprecated)
- * {@link #putTable(org.apache.hadoop.hbase.client.HTableInterface)}.
+ * {@link #putTable(org.apache.hadoop.hbase.client.Table)}.
*
*
* A pool can be created with a maxSize which defines the most HTable
@@ -78,18 +77,17 @@ import com.google.protobuf.ServiceException;
*
*
* Pool will manage its own connections to the cluster. See
- * {@link org.apache.hadoop.hbase.client.HConnectionManager}.
+ * {@link org.apache.hadoop.hbase.client.ConnectionManager}.
* Was @deprecated made @InterfaceAudience.private as of 0.98.1.
* See {@link org.apache.hadoop.hbase.client.HConnection#getTable(String)},
* Moved to thrift2 module for 2.0
*/
@InterfaceAudience.Private
public class HTablePool implements Closeable {
- private final PoolMap tables;
+ private final PoolMap tables;
private final int maxSize;
private final PoolType poolType;
private final Configuration config;
- private final HTableInterfaceFactory tableFactory;
/**
* Default Constructor. Default HBaseConfiguration and no limit on pool size.
@@ -99,18 +97,6 @@ public class HTablePool implements Closeable {
}
/**
- * Constructor to set maximum versions and use the specified configuration.
- *
- * @param config
- * configuration
- * @param maxSize
- * maximum number of references to keep for each table
- */
- public HTablePool(final Configuration config, final int maxSize) {
- this(config, maxSize, null, null);
- }
-
- /**
* Constructor to set maximum versions and use the specified configuration and
* table factory.
*
@@ -118,29 +104,9 @@ public class HTablePool implements Closeable {
* configuration
* @param maxSize
* maximum number of references to keep for each table
- * @param tableFactory
- * table factory
- */
- public HTablePool(final Configuration config, final int maxSize,
- final HTableInterfaceFactory tableFactory) {
- this(config, maxSize, tableFactory, PoolType.Reusable);
- }
-
- /**
- * Constructor to set maximum versions and use the specified configuration and
- * pool type.
- *
- * @param config
- * configuration
- * @param maxSize
- * maximum number of references to keep for each table
- * @param poolType
- * pool type which is one of {@link PoolType#Reusable} or
- * {@link PoolType#ThreadLocal}
*/
- public HTablePool(final Configuration config, final int maxSize,
- final PoolType poolType) {
- this(config, maxSize, null, poolType);
+ public HTablePool(final Configuration config, final int maxSize) {
+ this(config, maxSize, PoolType.Reusable);
}
/**
@@ -154,20 +120,16 @@ public class HTablePool implements Closeable {
* configuration
* @param maxSize
* maximum number of references to keep for each table
- * @param tableFactory
- * table factory
* @param poolType
* pool type which is one of {@link PoolType#Reusable} or
* {@link PoolType#ThreadLocal}
*/
public HTablePool(final Configuration config, final int maxSize,
- final HTableInterfaceFactory tableFactory, PoolType poolType) {
+ PoolType poolType) {
// Make a new configuration instance so I can safely cleanup when
// done with the pool.
this.config = config == null ? HBaseConfiguration.create() : config;
this.maxSize = maxSize;
- this.tableFactory = tableFactory == null ? new HTableFactory()
- : tableFactory;
if (poolType == null) {
this.poolType = PoolType.Reusable;
} else {
@@ -181,7 +143,7 @@ public class HTablePool implements Closeable {
break;
}
}
- this.tables = new PoolMap(this.poolType,
+ this.tables = new PoolMap<>(this.poolType,
this.maxSize);
}
@@ -196,9 +158,9 @@ public class HTablePool implements Closeable {
* @throws RuntimeException
* if there is a problem instantiating the HTable
*/
- public HTableInterface getTable(String tableName) {
+ public Table getTable(String tableName) {
// call the old getTable implementation renamed to findOrCreateTable
- HTableInterface table = findOrCreateTable(tableName);
+ Table table = findOrCreateTable(tableName);
// return a proxy table so when user closes the proxy, the actual table
// will be returned to the pool
return new PooledHTable(table);
@@ -216,8 +178,8 @@ public class HTablePool implements Closeable {
* @throws RuntimeException
* if there is a problem instantiating the HTable
*/
- private HTableInterface findOrCreateTable(String tableName) {
- HTableInterface table = tables.get(tableName);
+ private Table findOrCreateTable(String tableName) {
+ Table table = tables.get(tableName);
if (table == null) {
table = createHTable(tableName);
}
@@ -235,7 +197,7 @@ public class HTablePool implements Closeable {
* @return a reference to the specified table
* @throws RuntimeException if there is a problem instantiating the HTable
*/
- public HTableInterface getTable(byte[] tableName) {
+ public Table getTable(byte[] tableName) {
return getTable(Bytes.toString(tableName));
}
@@ -248,7 +210,7 @@ public class HTablePool implements Closeable {
* @deprecated
*/
@Deprecated
- public void putTable(HTableInterface table) throws IOException {
+ public void putTable(Table table) throws IOException {
// we need to be sure nobody puts a proxy implementation in the pool
// but if the client code is not updated
// and it will continue to call putTable() instead of calling close()
@@ -276,21 +238,25 @@ public class HTablePool implements Closeable {
* @param table
* table
*/
- private void returnTable(HTableInterface table) throws IOException {
+ private void returnTable(Table table) throws IOException {
// this is the old putTable method renamed and made private
- String tableName = Bytes.toString(table.getTableName());
+ String tableName = Bytes.toString(table.getName().toBytes());
if (tables.size(tableName) >= maxSize) {
// release table instance since we're not reusing it
this.tables.removeValue(tableName, table);
- this.tableFactory.releaseHTableInterface(table);
+ table.close();
return;
}
tables.put(tableName, table);
}
- protected HTableInterface createHTable(String tableName) {
- return this.tableFactory.createHTableInterface(config,
- Bytes.toBytes(tableName));
+ protected Table createHTable(String tableName) {
+ try {
+ return ConnectionFactory.createConnection(config).getTable(TableName.valueOf(tableName));
+ } catch (IOException e) {
+ //TODO: better way to handle that?
+ return null;
+ }
}
/**
@@ -298,16 +264,16 @@ public class HTablePool implements Closeable {
* table pool.
*
* Note: this is a 'shutdown' of the given table pool and different from
- * {@link #putTable(HTableInterface)}, that is used to return the table
+ * {@link #putTable(Table)}, that is used to return the table
* instance to the pool for future re-use.
*
* @param tableName
*/
public void closeTablePool(final String tableName) throws IOException {
- Collection tables = this.tables.values(tableName);
+ Collection tables = this.tables.values(tableName);
if (tables != null) {
- for (HTableInterface table : tables) {
- this.tableFactory.releaseHTableInterface(table);
+ for (Table table : tables) {
+ table.close();
}
}
this.tables.remove(tableName);
@@ -340,28 +306,22 @@ public class HTablePool implements Closeable {
}
/**
- * A proxy class that implements HTableInterface.close method to return the
+ * A proxy class that implements Table.close method to return the
* wrapped table back to the table pool
*
*/
- class PooledHTable implements HTableInterface {
+ class PooledHTable implements Table {
private boolean open = false;
- private HTableInterface table; // actual table implementation
+ private Table table; // actual table implementation
- public PooledHTable(HTableInterface table) {
+ public PooledHTable(Table table) {
this.table = table;
this.open = true;
}
@Override
- public byte[] getTableName() {
- checkState();
- return table.getTableName();
- }
-
- @Override
public TableName getName() {
return table.getName();
}
@@ -391,12 +351,6 @@ public class HTablePool implements Closeable {
}
@Override
- public Boolean[] exists(List gets) throws IOException {
- checkState();
- return table.exists(gets);
- }
-
- @Override
public void batch(List extends Row> actions, Object[] results) throws IOException,
InterruptedException {
checkState();
@@ -429,14 +383,6 @@ public class HTablePool implements Closeable {
}
@Override
- @SuppressWarnings("deprecation")
- @Deprecated
- public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
- checkState();
- return table.getRowOrBefore(row, family);
- }
-
- @Override
public ResultScanner getScanner(Scan scan) throws IOException {
checkState();
return table.getScanner(scan);
@@ -528,18 +474,6 @@ public class HTablePool implements Closeable {
durability);
}
- @Override
- public boolean isAutoFlush() {
- checkState();
- return table.isAutoFlush();
- }
-
- @Override
- public void flushCommits() throws IOException {
- checkState();
- table.flushCommits();
- }
-
/**
* Returns the actual table back to the pool
*
@@ -581,9 +515,9 @@ public class HTablePool implements Closeable {
/**
* Expose the wrapped HTable to tests in the same package
*
- * @return wrapped htable
+ * @return wrapped table
*/
- HTableInterface getWrappedTable() {
+ Table getWrappedTable() {
return table;
}
@@ -623,23 +557,6 @@ public class HTablePool implements Closeable {
}
@Override
- public void setAutoFlush(boolean autoFlush) {
- checkState();
- table.setAutoFlush(autoFlush, autoFlush);
- }
-
- @Override
- public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
- checkState();
- table.setAutoFlush(autoFlush, clearBufferOnFail);
- }
-
- @Override
- public void setAutoFlushTo(boolean autoFlush) {
- table.setAutoFlushTo(autoFlush);
- }
-
- @Override
public long getWriteBufferSize() {
checkState();
return table.getWriteBufferSize();
@@ -663,12 +580,6 @@ public class HTablePool implements Closeable {
}
@Override
- public long incrementColumnValue(byte[] row, byte[] family,
- byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
- return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
- }
-
- @Override
public Map batchCoprocessorService(
Descriptors.MethodDescriptor method, Message request,
byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable {
diff --git hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
index 41305a6..a7dc251 100644
--- hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
+++ hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
@@ -40,14 +40,11 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HTableFactory;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.thrift.ThriftMetrics;
import org.apache.hadoop.hbase.thrift2.generated.*;
-import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConnectionCache;
import org.apache.thrift.TException;
@@ -74,7 +71,6 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface {
new ConcurrentHashMap();
private final ConnectionCache connectionCache;
- private final HTableFactory tableFactory;
private final int maxPoolSize;
static final String CLEANUP_INTERVAL = "hbase.thrift.connection.cleanup-interval";
@@ -122,23 +118,12 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface {
int maxIdleTime = conf.getInt(MAX_IDLETIME, 10 * 60 * 1000);
connectionCache = new ConnectionCache(
conf, userProvider, cleanInterval, maxIdleTime);
- tableFactory = new HTableFactory() {
- @Override
- public HTableInterface createHTableInterface(Configuration config,
- byte[] tableName) {
- try {
- return connectionCache.getTable(Bytes.toString(tableName));
- } catch (IOException ioe) {
- throw new RuntimeException(ioe);
- }
- }
- };
htablePools = CacheBuilder.newBuilder().expireAfterAccess(
maxIdleTime, TimeUnit.MILLISECONDS).softValues().concurrencyLevel(4).build();
maxPoolSize = conf.getInt("hbase.thrift.htablepool.size.max", 1000);
htablePoolCreater = new Callable() {
public HTablePool call() {
- return new HTablePool(conf, maxPoolSize, tableFactory);
+ return new HTablePool(conf, maxPoolSize);
}
};
}
diff --git hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestHTablePool.java hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestHTablePool.java
index 101a7cf..bb561c6 100644
--- hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestHTablePool.java
+++ hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestHTablePool.java
@@ -22,8 +22,6 @@ import java.io.IOException;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -150,7 +148,7 @@ public class TestHTablePool {
// empty pool
// get table will return proxy implementation
- HTableInterface table = pool.getTable(tableName);
+ Table table = pool.getTable(tableName);
// put back the proxy implementation instead of closing it
pool.putTable(table);
@@ -172,12 +170,8 @@ public class TestHTablePool {
// empty pool
// get table will return proxy implementation
- final Table table = pool.getTable(tableName);
- HTableInterface alienTable = new HTable(TEST_UTIL.getConfiguration(),
- TableName.valueOf(TABLENAME)) {
- // implementation doesn't matter as long the table is not from
- // pool
- };
+ pool.getTable(tableName);
+ Table alienTable = TEST_UTIL.getConnection().getTable(TableName.valueOf(TABLENAME));
try {
// put the wrong table in pool
pool.putTable(alienTable);
|