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/client/ConnectionFactory.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index 89378dd..4b5ba79 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 */ @@ -222,7 +221,7 @@ public class ConnectionFactory { final ExecutorService pool, final User user) throws IOException { String className = conf.get(HConnection.HBASE_CLIENT_CONNECTION_IMPL, - ConnectionManager.HConnectionImplementation.class.getName()); + HConnectionImplementation.class.getName()); Class clazz = null; try { clazz = Class.forName(className); 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 50e2755..df26bf9 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 @@ -18,173 +18,34 @@ */ package org.apache.hadoop.hbase.client; -import javax.annotation.Nullable; - -import java.io.Closeable; import java.io.IOException; -import java.io.InterruptedIOException; -import java.lang.reflect.UndeclaredThrowableException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.util.ArrayList; import java.util.Date; -import java.util.HashSet; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.NavigableMap; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.BlockingRpcChannel; -import com.google.protobuf.RpcController; -import com.google.protobuf.ServiceException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableNotEnabledException; -import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture; -import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; -import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory; -import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.exceptions.RegionMovedException; import org.apache.hadoop.hbase.exceptions.RegionOpeningException; -import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.RequestConverter; -import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse; -import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ExceptionUtil; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.ipc.RemoteException; -import org.apache.zookeeper.KeeperException; /** * An internal, non-instantiable class that manages creation of {@link HConnection}s. @@ -196,7 +57,6 @@ final class ConnectionManager { static final Log LOG = LogFactory.getLog(ConnectionManager.class); public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server"; - private static final String CLIENT_NONCES_ENABLED_KEY = "hbase.client.nonces.enabled"; // An LRU Map of HConnectionKey -> HConnection (TableServer). All // access must be synchronized. This map is not private because tests @@ -205,14 +65,6 @@ final class ConnectionManager { public static final int MAX_CACHED_CONNECTION_INSTANCES; - /** - * Global nonceGenerator shared per client.Currently there's no reason to limit its scope. - * Once it's set under nonceGeneratorCreateLock, it is never unset or changed. - */ - private static volatile NonceGenerator nonceGenerator = null; - /** The nonce generator lock. Only taken when creating HConnection, which gets a private copy. */ - private static Object nonceGeneratorCreateLock = new Object(); - static { // We set instances to one more than the value specified for {@link // HConstants#ZOOKEEPER_MAX_CLIENT_CNXNS}. By default, the zk default max @@ -250,21 +102,6 @@ final class ConnectionManager { } /** - * @param conn The connection for which to replace the generator. - * @param cnm Replaces the nonce generator used, for testing. - * @return old nonce generator. - */ - @VisibleForTesting - static NonceGenerator injectNonceGeneratorForTesting( - ClusterConnection conn, NonceGenerator cnm) { - HConnectionImplementation connImpl = (HConnectionImplementation)conn; - NonceGenerator ng = connImpl.getNonceGenerator(); - LOG.warn("Nonce generator is being replaced by test code for " + cnm.getClass().getName()); - connImpl.nonceGenerator = cnm; - return ng; - } - - /** * 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 @@ -335,7 +172,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 +198,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 +223,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(...); * ... @@ -425,19 +262,6 @@ final class ConnectionManager { } /** - * 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) { - deleteConnection(new HConnectionKey(conf), false); - } - - /** * Cleanup a known stale connection. * This will then close connection to the zookeeper ensemble and let go of all resources. * @@ -450,37 +274,10 @@ final class ConnectionManager { } /** - * 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) { - synchronized (CONNECTION_INSTANCES) { - Set connectionKeys = new HashSet(); - connectionKeys.addAll(CONNECTION_INSTANCES.keySet()); - for (HConnectionKey connectionKey : connectionKeys) { - deleteConnection(connectionKey, staleConnection); - } - CONNECTION_INSTANCES.clear(); - } - } - - /** - * Delete information for all connections.. - * @deprecated kept for backward compatibility, but the behavior is broken. HBASE-8983 - */ - @Deprecated - public static void deleteAllConnections() { - deleteAllConnections(false); - } - - /** * @deprecated connection caching is going away. */ @Deprecated - private static void deleteConnection(HConnection connection, boolean staleConnection) { + static void deleteConnection(HConnection connection, boolean staleConnection) { synchronized (CONNECTION_INSTANCES) { for (Entry e: CONNECTION_INSTANCES.entrySet()) { if (e.getValue() == connection) { @@ -547,2031 +344,6 @@ final class ConnectionManager { } } - /** Encapsulates connection to zookeeper and regionservers.*/ - @edu.umd.cs.findbugs.annotations.SuppressWarnings( - value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION", - justification="Access to the conncurrent hash map is under a lock so should be fine.") - static class HConnectionImplementation implements ClusterConnection, Closeable { - static final Log LOG = LogFactory.getLog(HConnectionImplementation.class); - private final long pause; - private final boolean useMetaReplicas; - private final int numTries; - final int rpcTimeout; - private NonceGenerator nonceGenerator = null; - private final AsyncProcess asyncProcess; - // single tracker per connection - private final ServerStatisticTracker stats; - - private volatile boolean closed; - private volatile boolean aborted; - - // package protected for the tests - ClusterStatusListener clusterStatusListener; - - - private final Object metaRegionLock = new Object(); - - // We have a single lock for master & zk to prevent deadlocks. Having - // one lock for ZK and one lock for master is not possible: - // When creating a connection to master, we need a connection to ZK to get - // its address. But another thread could have taken the ZK lock, and could - // be waiting for the master lock => deadlock. - private final Object masterAndZKLock = new Object(); - - private long keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE; - - // thread executor shared by all HTableInterface instances created - // by this connection - private volatile ExecutorService batchPool = null; - // meta thread executor shared by all HTableInterface instances created - // by this connection - private volatile ExecutorService metaLookupPool = null; - private volatile boolean cleanupPool = false; - - private final Configuration conf; - - // cache the configuration value for tables so that we can avoid calling - // the expensive Configuration to fetch the value multiple times. - private final TableConfiguration tableConfig; - - // Client rpc instance. - private RpcClient rpcClient; - - private MetaCache metaCache = new MetaCache(); - - private int refCount; - - // indicates whether this connection's life cycle is managed (by us) - private boolean managed; - - private User user; - - private RpcRetryingCallerFactory rpcCallerFactory; - - private RpcControllerFactory rpcControllerFactory; - - private final RetryingCallerInterceptor interceptor; - - /** - * Cluster registry of basic info such as clusterid and meta region location. - */ - Registry registry; - - private final ClientBackoffPolicy backoffPolicy; - - HConnectionImplementation(Configuration conf, boolean managed) throws IOException { - this(conf, managed, null, null); - } - - /** - * constructor - * @param conf Configuration object - * @param managed If true, does not do full shutdown on close; i.e. cleanup of connection - * to zk and shutdown of all services; we just close down the resources this connection was - * responsible for and decrement usage counters. It is up to the caller to do the full - * cleanup. It is set when we want have connection sharing going on -- reuse of zk connection, - * and cached region locations, established regionserver connections, etc. When connections - * are shared, we have reference counting going on and will only do full cleanup when no more - * users of an HConnectionImplementation instance. - */ - HConnectionImplementation(Configuration conf, boolean managed, - ExecutorService pool, User user) throws IOException { - this(conf); - this.user = user; - this.batchPool = pool; - this.managed = managed; - this.registry = setupRegistry(); - retrieveClusterId(); - - this.rpcClient = RpcClientFactory.createClient(this.conf, this.clusterId); - this.rpcControllerFactory = RpcControllerFactory.instantiate(conf); - - // Do we publish the status? - boolean shouldListen = conf.getBoolean(HConstants.STATUS_PUBLISHED, - HConstants.STATUS_PUBLISHED_DEFAULT); - Class listenerClass = - conf.getClass(ClusterStatusListener.STATUS_LISTENER_CLASS, - ClusterStatusListener.DEFAULT_STATUS_LISTENER_CLASS, - ClusterStatusListener.Listener.class); - if (shouldListen) { - if (listenerClass == null) { - LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " + - ClusterStatusListener.STATUS_LISTENER_CLASS + " is not set - not listening status"); - } else { - clusterStatusListener = new ClusterStatusListener( - new ClusterStatusListener.DeadServerHandler() { - @Override - public void newDead(ServerName sn) { - clearCaches(sn); - rpcClient.cancelConnections(sn); - } - }, conf, listenerClass); - } - } - } - - /** - * For tests. - */ - protected HConnectionImplementation(Configuration conf) { - this.conf = conf; - this.tableConfig = new TableConfiguration(conf); - this.closed = false; - this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, - HConstants.DEFAULT_HBASE_CLIENT_PAUSE); - this.useMetaReplicas = conf.getBoolean(HConstants.USE_META_REPLICAS, - HConstants.DEFAULT_USE_META_REPLICAS); - this.numTries = tableConfig.getRetriesNumber(); - this.rpcTimeout = conf.getInt( - HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT); - if (conf.getBoolean(CLIENT_NONCES_ENABLED_KEY, true)) { - synchronized (nonceGeneratorCreateLock) { - if (ConnectionManager.nonceGenerator == null) { - ConnectionManager.nonceGenerator = new PerClientRandomNonceGenerator(); - } - this.nonceGenerator = ConnectionManager.nonceGenerator; - } - } else { - this.nonceGenerator = new NoNonceGenerator(); - } - stats = ServerStatisticTracker.create(conf); - this.asyncProcess = createAsyncProcess(this.conf); - this.interceptor = (new RetryingCallerInterceptorFactory(conf)).build(); - this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, interceptor, this.stats); - this.backoffPolicy = ClientBackoffPolicyFactory.create(conf); - } - - @Override - public HTableInterface getTable(String tableName) throws IOException { - return getTable(TableName.valueOf(tableName)); - } - - @Override - public HTableInterface getTable(byte[] tableName) throws IOException { - return getTable(TableName.valueOf(tableName)); - } - - @Override - public HTableInterface getTable(TableName tableName) throws IOException { - return getTable(tableName, getBatchPool()); - } - - @Override - public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException { - return getTable(TableName.valueOf(tableName), pool); - } - - @Override - public HTableInterface getTable(byte[] tableName, ExecutorService pool) throws IOException { - return getTable(TableName.valueOf(tableName), pool); - } - - @Override - public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException { - if (managed) { - throw new NeedUnmanagedConnectionException(); - } - return new HTable(tableName, this, tableConfig, rpcCallerFactory, rpcControllerFactory, pool); - } - - @Override - public BufferedMutator getBufferedMutator(BufferedMutatorParams params) { - if (params.getTableName() == null) { - throw new IllegalArgumentException("TableName cannot be null."); - } - if (params.getPool() == null) { - params.pool(HTable.getDefaultExecutor(getConfiguration())); - } - if (params.getWriteBufferSize() == BufferedMutatorParams.UNSET) { - params.writeBufferSize(tableConfig.getWriteBufferSize()); - } - if (params.getMaxKeyValueSize() == BufferedMutatorParams.UNSET) { - params.maxKeyValueSize(tableConfig.getMaxKeyValueSize()); - } - return new BufferedMutatorImpl(this, rpcCallerFactory, rpcControllerFactory, params); - } - - @Override - public BufferedMutator getBufferedMutator(TableName tableName) { - return getBufferedMutator(new BufferedMutatorParams(tableName)); - } - - @Override - public RegionLocator getRegionLocator(TableName tableName) throws IOException { - return new HRegionLocator(tableName, this); - } - - @Override - public Admin getAdmin() throws IOException { - if (managed) { - throw new NeedUnmanagedConnectionException(); - } - return new HBaseAdmin(this); - } - - private ExecutorService getBatchPool() { - if (batchPool == null) { - synchronized (this) { - if (batchPool == null) { - this.batchPool = getThreadPool(conf.getInt("hbase.hconnection.threads.max", 256), - conf.getInt("hbase.hconnection.threads.core", 256), "-shared-"); - this.cleanupPool = true; - } - } - } - return this.batchPool; - } - - private ExecutorService getThreadPool(int maxThreads, int coreThreads, String nameHint) { - // shared HTable thread executor not yet initialized - if (maxThreads == 0) { - maxThreads = Runtime.getRuntime().availableProcessors() * 8; - } - if (coreThreads == 0) { - coreThreads = Runtime.getRuntime().availableProcessors() * 8; - } - long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 60); - LinkedBlockingQueue workQueue = - new LinkedBlockingQueue(maxThreads * - conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, - HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS)); - ThreadPoolExecutor tpe = new ThreadPoolExecutor( - coreThreads, - maxThreads, - keepAliveTime, - TimeUnit.SECONDS, - workQueue, - Threads.newDaemonThreadFactory(toString() + nameHint)); - tpe.allowCoreThreadTimeOut(true); - return tpe; - } - - private ExecutorService getMetaLookupPool() { - if (this.metaLookupPool == null) { - synchronized (this) { - if (this.metaLookupPool == null) { - //The meta lookup can happen on replicas of the meta (if the appropriate configs - //are enabled).In a replicated-meta setup, the number '3' is assumed as the max - //number of replicas by default (unless it is configured to be of a higher value). - //In a non-replicated-meta setup, only one thread would be active. - this.metaLookupPool = getThreadPool( - conf.getInt("hbase.hconnection.meta.lookup.threads.max", 3), - conf.getInt("hbase.hconnection.meta.lookup.threads.max.core", 3), - "-metaLookup-shared-"); - } - } - } - return this.metaLookupPool; - } - - protected ExecutorService getCurrentMetaLookupPool() { - return metaLookupPool; - } - - protected ExecutorService getCurrentBatchPool() { - return batchPool; - } - - private void shutdownPools() { - if (this.cleanupPool && this.batchPool != null && !this.batchPool.isShutdown()) { - shutdownBatchPool(this.batchPool); - } - if (this.metaLookupPool != null && !this.metaLookupPool.isShutdown()) { - shutdownBatchPool(this.metaLookupPool); - } - } - - private void shutdownBatchPool(ExecutorService pool) { - pool.shutdown(); - try { - if (!pool.awaitTermination(10, TimeUnit.SECONDS)) { - pool.shutdownNow(); - } - } catch (InterruptedException e) { - pool.shutdownNow(); - } - } - - /** - * @return The cluster registry implementation to use. - * @throws IOException - */ - private Registry setupRegistry() throws IOException { - return RegistryFactory.getRegistry(this); - } - - /** - * For tests only. - */ - @VisibleForTesting - RpcClient getRpcClient() { - return rpcClient; - } - - /** - * An identifier that will remain the same for a given connection. - */ - @Override - public String toString(){ - return "hconnection-0x" + Integer.toHexString(hashCode()); - } - - protected String clusterId = null; - - void retrieveClusterId() { - if (clusterId != null) return; - this.clusterId = this.registry.getClusterId(); - if (clusterId == null) { - clusterId = HConstants.CLUSTER_ID_DEFAULT; - LOG.debug("clusterid came back null, using default " + clusterId); - } - } - - @Override - public Configuration getConfiguration() { - return this.conf; - } - - private void checkIfBaseNodeAvailable(ZooKeeperWatcher zkw) - throws MasterNotRunningException { - String errorMsg; - try { - if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) { - errorMsg = "The node " + zkw.baseZNode+" is not in ZooKeeper. " - + "It should have been written by the master. " - + "Check the value configured in 'zookeeper.znode.parent'. " - + "There could be a mismatch with the one configured in the master."; - LOG.error(errorMsg); - throw new MasterNotRunningException(errorMsg); - } - } catch (KeeperException e) { - errorMsg = "Can't get connection to ZooKeeper: " + e.getMessage(); - LOG.error(errorMsg); - throw new MasterNotRunningException(errorMsg, e); - } - } - - /** - * @return true if the master is running, throws an exception otherwise - * @throws MasterNotRunningException - if the master is not running - * @throws ZooKeeperConnectionException - * @deprecated this has been deprecated without a replacement - */ - @Deprecated - @Override - public boolean isMasterRunning() - throws MasterNotRunningException, ZooKeeperConnectionException { - // When getting the master connection, we check it's running, - // so if there is no exception, it means we've been able to get a - // connection on a running master - MasterKeepAliveConnection m = getKeepAliveMasterService(); - m.close(); - return true; - } - - @Override - public HRegionLocation getRegionLocation(final TableName tableName, - final byte [] row, boolean reload) - throws IOException { - return reload? relocateRegion(tableName, row): locateRegion(tableName, row); - } - - @Override - public HRegionLocation getRegionLocation(final byte[] tableName, - final byte [] row, boolean reload) - throws IOException { - return getRegionLocation(TableName.valueOf(tableName), row, reload); - } - - @Override - public boolean isTableEnabled(TableName tableName) throws IOException { - return getTableState(tableName).inStates(TableState.State.ENABLED); - } - - @Override - public boolean isTableEnabled(byte[] tableName) throws IOException { - return isTableEnabled(TableName.valueOf(tableName)); - } - - @Override - public boolean isTableDisabled(TableName tableName) throws IOException { - return getTableState(tableName).inStates(TableState.State.DISABLED); - } - - @Override - public boolean isTableDisabled(byte[] tableName) throws IOException { - return isTableDisabled(TableName.valueOf(tableName)); - } - - @Override - public boolean isTableAvailable(final TableName tableName) throws IOException { - return isTableAvailable(tableName, null); - } - - @Override - public boolean isTableAvailable(final byte[] tableName) throws IOException { - return isTableAvailable(TableName.valueOf(tableName)); - } - - @Override - public boolean isTableAvailable(final TableName tableName, @Nullable final byte[][] splitKeys) - throws IOException { - try { - if (!isTableEnabled(tableName)) { - LOG.debug("Table " + tableName + " not enabled"); - return false; - } - ClusterConnection connection = getConnectionInternal(getConfiguration()); - List> locations = MetaTableAccessor - .getTableRegionsAndLocations(connection, tableName, true); - int notDeployed = 0; - int regionCount = 0; - for (Pair pair : locations) { - HRegionInfo info = pair.getFirst(); - if (pair.getSecond() == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Table " + tableName + " has not deployed region " + pair.getFirst() - .getEncodedName()); - } - notDeployed++; - } else if (splitKeys != null - && !Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) { - for (byte[] splitKey : splitKeys) { - // Just check if the splitkey is available - if (Bytes.equals(info.getStartKey(), splitKey)) { - regionCount++; - break; - } - } - } else { - // Always empty start row should be counted - regionCount++; - } - } - if (notDeployed > 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("Table " + tableName + " has " + notDeployed + " regions"); - } - return false; - } else if (splitKeys != null && regionCount != splitKeys.length + 1) { - if (LOG.isDebugEnabled()) { - LOG.debug("Table " + tableName + " expected to have " + (splitKeys.length + 1) - + " regions, but only " + regionCount + " available"); - } - return false; - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Table " + tableName + " should be available"); - } - return true; - } - } catch (TableNotFoundException tnfe) { - LOG.warn("Table " + tableName + " not enabled, it is not exists"); - return false; - } - } - - @Override - public boolean isTableAvailable(final byte[] tableName, final byte[][] splitKeys) - throws IOException { - return isTableAvailable(TableName.valueOf(tableName), splitKeys); - } - - @Override - public HRegionLocation locateRegion(final byte[] regionName) throws IOException { - RegionLocations locations = locateRegion(HRegionInfo.getTable(regionName), - HRegionInfo.getStartKey(regionName), false, true); - return locations == null ? null : locations.getRegionLocation(); - } - - @Override - public boolean isDeadServer(ServerName sn) { - if (clusterStatusListener == null) { - return false; - } else { - return clusterStatusListener.isDeadServer(sn); - } - } - - @Override - public List locateRegions(final TableName tableName) - throws IOException { - return locateRegions(tableName, false, true); - } - - @Override - public List locateRegions(final byte[] tableName) - throws IOException { - return locateRegions(TableName.valueOf(tableName)); - } - - @Override - public List locateRegions(final TableName tableName, - final boolean useCache, final boolean offlined) throws IOException { - List regions = MetaTableAccessor - .getTableRegions(this, tableName, !offlined); - final List locations = new ArrayList(); - for (HRegionInfo regionInfo : regions) { - RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true); - if (list != null) { - for (HRegionLocation loc : list.getRegionLocations()) { - if (loc != null) { - locations.add(loc); - } - } - } - } - return locations; - } - - @Override - public List locateRegions(final byte[] tableName, - final boolean useCache, final boolean offlined) throws IOException { - return locateRegions(TableName.valueOf(tableName), useCache, offlined); - } - - @Override - public HRegionLocation locateRegion( - final TableName tableName, final byte[] row) throws IOException{ - RegionLocations locations = locateRegion(tableName, row, true, true); - return locations == null ? null : locations.getRegionLocation(); - } - - @Override - public HRegionLocation locateRegion(final byte[] tableName, - final byte [] row) - throws IOException{ - return locateRegion(TableName.valueOf(tableName), row); - } - - @Override - public HRegionLocation relocateRegion(final TableName tableName, - final byte [] row) throws IOException{ - RegionLocations locations = relocateRegion(tableName, row, - RegionReplicaUtil.DEFAULT_REPLICA_ID); - return locations == null ? null : - locations.getRegionLocation(RegionReplicaUtil.DEFAULT_REPLICA_ID); - } - - @Override - public RegionLocations relocateRegion(final TableName tableName, - final byte [] row, int replicaId) throws IOException{ - // Since this is an explicit request not to use any caching, finding - // disabled tables should not be desirable. This will ensure that an exception is thrown when - // the first time a disabled table is interacted with. - if (!tableName.equals(TableName.META_TABLE_NAME) && isTableDisabled(tableName)) { - throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled."); - } - - return locateRegion(tableName, row, false, true, replicaId); - } - - @Override - public HRegionLocation relocateRegion(final byte[] tableName, - final byte [] row) throws IOException { - return relocateRegion(TableName.valueOf(tableName), row); - } - - @Override - public RegionLocations locateRegion(final TableName tableName, - final byte [] row, boolean useCache, boolean retry) - throws IOException { - return locateRegion(tableName, row, useCache, retry, RegionReplicaUtil.DEFAULT_REPLICA_ID); - } - - @Override - public RegionLocations locateRegion(final TableName tableName, - final byte [] row, boolean useCache, boolean retry, int replicaId) - throws IOException { - if (this.closed) throw new IOException(toString() + " closed"); - if (tableName== null || tableName.getName().length == 0) { - throw new IllegalArgumentException( - "table name cannot be null or zero length"); - } - if (tableName.equals(TableName.META_TABLE_NAME)) { - return locateMeta(tableName, useCache, replicaId); - } else { - // Region not in the cache - have to go to the meta RS - return locateRegionInMeta(tableName, row, useCache, retry, replicaId); - } - } - - private RegionLocations locateMeta(final TableName tableName, - boolean useCache, int replicaId) throws IOException { - // HBASE-10785: We cache the location of the META itself, so that we are not overloading - // zookeeper with one request for every region lookup. We cache the META with empty row - // key in MetaCache. - byte[] metaCacheKey = HConstants.EMPTY_START_ROW; // use byte[0] as the row for meta - RegionLocations locations = null; - if (useCache) { - locations = getCachedLocation(tableName, metaCacheKey); - if (locations != null && locations.getRegionLocation(replicaId) != null) { - return locations; - } - } - - // only one thread should do the lookup. - synchronized (metaRegionLock) { - // Check the cache again for a hit in case some other thread made the - // same query while we were waiting on the lock. - if (useCache) { - locations = getCachedLocation(tableName, metaCacheKey); - if (locations != null && locations.getRegionLocation(replicaId) != null) { - return locations; - } - } - - // Look up from zookeeper - locations = this.registry.getMetaRegionLocation(); - if (locations != null) { - cacheLocation(tableName, locations); - } - } - return locations; - } - - /* - * Search the hbase:meta table for the HRegionLocation - * info that contains the table and row we're seeking. - */ - private RegionLocations locateRegionInMeta(TableName tableName, byte[] row, - boolean useCache, boolean retry, int replicaId) throws IOException { - - // If we are supposed to be using the cache, look in the cache to see if - // we already have the region. - if (useCache) { - RegionLocations locations = getCachedLocation(tableName, row); - if (locations != null && locations.getRegionLocation(replicaId) != null) { - return locations; - } - } - - // build the key of the meta region we should be looking for. - // the extra 9's on the end are necessary to allow "exact" matches - // without knowing the precise region names. - byte[] metaKey = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false); - - Scan s = new Scan(); - s.setReversed(true); - s.setStartRow(metaKey); - s.setSmall(true); - s.setCaching(1); - if (this.useMetaReplicas) { - s.setConsistency(Consistency.TIMELINE); - } - - int localNumRetries = (retry ? numTries : 1); - - for (int tries = 0; true; tries++) { - if (tries >= localNumRetries) { - throw new NoServerForRegionException("Unable to find region for " - + Bytes.toStringBinary(row) + " in " + tableName + - " after " + localNumRetries + " tries."); - } - if (useCache) { - RegionLocations locations = getCachedLocation(tableName, row); - if (locations != null && locations.getRegionLocation(replicaId) != null) { - return locations; - } - } else { - // If we are not supposed to be using the cache, delete any existing cached location - // so it won't interfere. - metaCache.clearCache(tableName, row); - } - - // Query the meta region - try { - Result regionInfoRow = null; - ReversedClientScanner rcs = null; - try { - rcs = new ClientSmallReversedScanner(conf, s, TableName.META_TABLE_NAME, this, - rpcCallerFactory, rpcControllerFactory, getMetaLookupPool(), 0); - regionInfoRow = rcs.next(); - } finally { - if (rcs != null) { - rcs.close(); - } - } - - if (regionInfoRow == null) { - throw new TableNotFoundException(tableName); - } - - // convert the row result into the HRegionLocation we need! - RegionLocations locations = MetaTableAccessor.getRegionLocations(regionInfoRow); - if (locations == null || locations.getRegionLocation(replicaId) == null) { - throw new IOException("HRegionInfo was null in " + - tableName + ", row=" + regionInfoRow); - } - HRegionInfo regionInfo = locations.getRegionLocation(replicaId).getRegionInfo(); - if (regionInfo == null) { - throw new IOException("HRegionInfo was null or empty in " + - TableName.META_TABLE_NAME + ", row=" + regionInfoRow); - } - - // possible we got a region of a different table... - if (!regionInfo.getTable().equals(tableName)) { - throw new TableNotFoundException( - "Table '" + tableName + "' was not found, got: " + - regionInfo.getTable() + "."); - } - if (regionInfo.isSplit()) { - throw new RegionOfflineException("the only available region for" + - " the required row is a split parent," + - " the daughters should be online soon: " + - regionInfo.getRegionNameAsString()); - } - if (regionInfo.isOffline()) { - throw new RegionOfflineException("the region is offline, could" + - " be caused by a disable table call: " + - regionInfo.getRegionNameAsString()); - } - - ServerName serverName = locations.getRegionLocation(replicaId).getServerName(); - if (serverName == null) { - throw new NoServerForRegionException("No server address listed " + - "in " + TableName.META_TABLE_NAME + " for region " + - regionInfo.getRegionNameAsString() + " containing row " + - Bytes.toStringBinary(row)); - } - - if (isDeadServer(serverName)){ - throw new RegionServerStoppedException("hbase:meta says the region "+ - regionInfo.getRegionNameAsString()+" is managed by the server " + serverName + - ", but it is dead."); - } - // Instantiate the location - cacheLocation(tableName, locations); - return locations; - } catch (TableNotFoundException e) { - // if we got this error, probably means the table just plain doesn't - // exist. rethrow the error immediately. this should always be coming - // from the HTable constructor. - throw e; - } catch (IOException e) { - ExceptionUtil.rethrowIfInterrupt(e); - - if (e instanceof RemoteException) { - e = ((RemoteException)e).unwrapRemoteException(); - } - if (tries < localNumRetries - 1) { - if (LOG.isDebugEnabled()) { - LOG.debug("locateRegionInMeta parentTable=" + - TableName.META_TABLE_NAME + ", metaLocation=" + - ", attempt=" + tries + " of " + - localNumRetries + " failed; retrying after sleep of " + - ConnectionUtils.getPauseTime(this.pause, tries) + " because: " + e.getMessage()); - } - } else { - throw e; - } - // Only relocate the parent region if necessary - if(!(e instanceof RegionOfflineException || - e instanceof NoServerForRegionException)) { - relocateRegion(TableName.META_TABLE_NAME, metaKey, replicaId); - } - } - try{ - Thread.sleep(ConnectionUtils.getPauseTime(this.pause, tries)); - } catch (InterruptedException e) { - throw new InterruptedIOException("Giving up trying to location region in " + - "meta: thread is interrupted."); - } - } - } - - /** - * Put a newly discovered HRegionLocation into the cache. - * @param tableName The table name. - * @param location the new location - */ - private void cacheLocation(final TableName tableName, final RegionLocations location) { - metaCache.cacheLocation(tableName, location); - } - - /** - * Search the cache for a location that fits our table and row key. - * Return null if no suitable region is located. - * - * @param tableName - * @param row - * @return Null or region location found in cache. - */ - RegionLocations getCachedLocation(final TableName tableName, - final byte [] row) { - return metaCache.getCachedLocation(tableName, row); - } - - public void clearRegionCache(final TableName tableName, byte[] row) { - metaCache.clearCache(tableName, row); - } - - /* - * Delete all cached entries of a table that maps to a specific location. - */ - @Override - public void clearCaches(final ServerName serverName) { - metaCache.clearCache(serverName); - } - - @Override - public void clearRegionCache() { - metaCache.clearCache(); - } - - @Override - public void clearRegionCache(final TableName tableName) { - metaCache.clearCache(tableName); - } - - @Override - public void clearRegionCache(final byte[] tableName) { - clearRegionCache(TableName.valueOf(tableName)); - } - - /** - * Put a newly discovered HRegionLocation into the cache. - * @param tableName The table name. - * @param source the source of the new location, if it's not coming from meta - * @param location the new location - */ - private void cacheLocation(final TableName tableName, final ServerName source, - final HRegionLocation location) { - metaCache.cacheLocation(tableName, source, location); - } - - // Map keyed by service name + regionserver to service stub implementation - private final ConcurrentHashMap stubs = - new ConcurrentHashMap(); - // Map of locks used creating service stubs per regionserver. - private final ConcurrentHashMap connectionLock = - new ConcurrentHashMap(); - - /** - * State of the MasterService connection/setup. - */ - static class MasterServiceState { - HConnection connection; - MasterService.BlockingInterface stub; - int userCount; - - MasterServiceState(final HConnection connection) { - super(); - this.connection = connection; - } - - @Override - public String toString() { - return "MasterService"; - } - - Object getStub() { - return this.stub; - } - - void clearStub() { - this.stub = null; - } - - boolean isMasterRunning() throws ServiceException { - IsMasterRunningResponse response = - this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest()); - return response != null? response.getIsMasterRunning(): false; - } - } - - /** - * Makes a client-side stub for master services. Sub-class to specialize. - * Depends on hosting class so not static. Exists so we avoid duplicating a bunch of code - * when setting up the MasterMonitorService and MasterAdminService. - */ - abstract class StubMaker { - /** - * Returns the name of the service stub being created. - */ - protected abstract String getServiceName(); - - /** - * Make stub and cache it internal so can be used later doing the isMasterRunning call. - * @param channel - */ - protected abstract Object makeStub(final BlockingRpcChannel channel); - - /** - * Once setup, check it works by doing isMasterRunning check. - * @throws ServiceException - */ - protected abstract void isMasterRunning() throws ServiceException; - - /** - * Create a stub. Try once only. It is not typed because there is no common type to - * protobuf services nor their interfaces. Let the caller do appropriate casting. - * @return A stub for master services. - * @throws IOException - * @throws KeeperException - * @throws ServiceException - */ - private Object makeStubNoRetries() throws IOException, KeeperException, ServiceException { - ZooKeeperKeepAliveConnection zkw; - try { - zkw = getKeepAliveZooKeeperWatcher(); - } catch (IOException e) { - ExceptionUtil.rethrowIfInterrupt(e); - throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e); - } - try { - checkIfBaseNodeAvailable(zkw); - ServerName sn = MasterAddressTracker.getMasterAddress(zkw); - if (sn == null) { - String msg = "ZooKeeper available but no active master location found"; - LOG.info(msg); - throw new MasterNotRunningException(msg); - } - if (isDeadServer(sn)) { - throw new MasterNotRunningException(sn + " is dead."); - } - // Use the security info interface name as our stub key - String key = getStubKey(getServiceName(), sn.getHostname(), sn.getPort()); - connectionLock.putIfAbsent(key, key); - Object stub = null; - synchronized (connectionLock.get(key)) { - stub = stubs.get(key); - if (stub == null) { - BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout); - stub = makeStub(channel); - isMasterRunning(); - stubs.put(key, stub); - } - } - return stub; - } finally { - zkw.close(); - } - } - - /** - * Create a stub against the master. Retry if necessary. - * @return A stub to do intf against the master - * @throws MasterNotRunningException - */ - Object makeStub() throws IOException { - // The lock must be at the beginning to prevent multiple master creations - // (and leaks) in a multithread context - synchronized (masterAndZKLock) { - Exception exceptionCaught = null; - if (!closed) { - try { - return makeStubNoRetries(); - } catch (IOException e) { - exceptionCaught = e; - } catch (KeeperException e) { - exceptionCaught = e; - } catch (ServiceException e) { - exceptionCaught = e; - } - - throw new MasterNotRunningException(exceptionCaught); - } else { - throw new DoNotRetryIOException("Connection was closed while trying to get master"); - } - } - } - } - - /** - * Class to make a MasterServiceStubMaker stub. - */ - class MasterServiceStubMaker extends StubMaker { - private MasterService.BlockingInterface stub; - @Override - protected String getServiceName() { - return MasterService.getDescriptor().getName(); - } - - @Override - MasterService.BlockingInterface makeStub() throws IOException { - return (MasterService.BlockingInterface)super.makeStub(); - } - - @Override - protected Object makeStub(BlockingRpcChannel channel) { - this.stub = MasterService.newBlockingStub(channel); - return this.stub; - } - - @Override - protected void isMasterRunning() throws ServiceException { - this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest()); - } - } - - @Override - public AdminService.BlockingInterface getAdmin(final ServerName serverName) - throws IOException { - return getAdmin(serverName, false); - } - - @Override - // Nothing is done w/ the 'master' parameter. It is ignored. - public AdminService.BlockingInterface getAdmin(final ServerName serverName, - final boolean master) - throws IOException { - if (isDeadServer(serverName)) { - throw new RegionServerStoppedException(serverName + " is dead."); - } - String key = getStubKey(AdminService.BlockingInterface.class.getName(), - serverName.getHostname(), serverName.getPort()); - this.connectionLock.putIfAbsent(key, key); - AdminService.BlockingInterface stub = null; - synchronized (this.connectionLock.get(key)) { - stub = (AdminService.BlockingInterface)this.stubs.get(key); - if (stub == null) { - BlockingRpcChannel channel = - this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout); - stub = AdminService.newBlockingStub(channel); - this.stubs.put(key, stub); - } - } - return stub; - } - - @Override - public ClientService.BlockingInterface getClient(final ServerName sn) - throws IOException { - if (isDeadServer(sn)) { - throw new RegionServerStoppedException(sn + " is dead."); - } - String key = getStubKey(ClientService.BlockingInterface.class.getName(), sn.getHostname(), - sn.getPort()); - this.connectionLock.putIfAbsent(key, key); - ClientService.BlockingInterface stub = null; - synchronized (this.connectionLock.get(key)) { - stub = (ClientService.BlockingInterface)this.stubs.get(key); - if (stub == null) { - BlockingRpcChannel channel = - this.rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout); - stub = ClientService.newBlockingStub(channel); - // In old days, after getting stub/proxy, we'd make a call. We are not doing that here. - // Just fail on first actual call rather than in here on setup. - this.stubs.put(key, stub); - } - } - return stub; - } - - static String getStubKey(final String serviceName, final String rsHostname, int port) { - // Sometimes, servers go down and they come back up with the same hostname but a different - // IP address. Force a resolution of the rsHostname by trying to instantiate an - // InetSocketAddress, and this way we will rightfully get a new stubKey. - // Also, include the hostname in the key so as to take care of those cases where the - // DNS name is different but IP address remains the same. - InetAddress i = new InetSocketAddress(rsHostname, port).getAddress(); - String address = rsHostname; - if (i != null) { - address = i.getHostAddress() + "-" + rsHostname; - } - return serviceName + "@" + address + ":" + port; - } - - private ZooKeeperKeepAliveConnection keepAliveZookeeper; - private AtomicInteger keepAliveZookeeperUserCount = new AtomicInteger(0); - private boolean canCloseZKW = true; - - // keepAlive time, in ms. No reason to make it configurable. - private static final long keepAlive = 5 * 60 * 1000; - - /** - * Retrieve a shared ZooKeeperWatcher. You must close it it once you've have finished with it. - * @return The shared instance. Never returns null. - */ - ZooKeeperKeepAliveConnection getKeepAliveZooKeeperWatcher() - throws IOException { - synchronized (masterAndZKLock) { - if (keepAliveZookeeper == null) { - if (this.closed) { - throw new IOException(toString() + " closed"); - } - // We don't check that our link to ZooKeeper is still valid - // But there is a retry mechanism in the ZooKeeperWatcher itself - keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this); - } - keepAliveZookeeperUserCount.addAndGet(1); - keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE; - return keepAliveZookeeper; - } - } - - void releaseZooKeeperWatcher(final ZooKeeperWatcher zkw) { - if (zkw == null){ - return; - } - if (keepAliveZookeeperUserCount.addAndGet(-1) <= 0) { - keepZooKeeperWatcherAliveUntil = System.currentTimeMillis() + keepAlive; - } - } - - private void closeZooKeeperWatcher() { - synchronized (masterAndZKLock) { - if (keepAliveZookeeper != null) { - LOG.info("Closing zookeeper sessionid=0x" + - Long.toHexString( - keepAliveZookeeper.getRecoverableZooKeeper().getSessionId())); - keepAliveZookeeper.internalClose(); - keepAliveZookeeper = null; - } - keepAliveZookeeperUserCount.set(0); - } - } - - final MasterServiceState masterServiceState = new MasterServiceState(this); - - @Override - public MasterService.BlockingInterface getMaster() throws MasterNotRunningException { - return getKeepAliveMasterService(); - } - - private void resetMasterServiceState(final MasterServiceState mss) { - mss.userCount++; - } - - @Override - public MasterKeepAliveConnection getKeepAliveMasterService() - throws MasterNotRunningException { - synchronized (masterAndZKLock) { - if (!isKeepAliveMasterConnectedAndRunning(this.masterServiceState)) { - MasterServiceStubMaker stubMaker = new MasterServiceStubMaker(); - try { - this.masterServiceState.stub = stubMaker.makeStub(); - } catch (MasterNotRunningException ex) { - throw ex; - } catch (IOException e) { - // rethrow as MasterNotRunningException so that we can keep the method sig - throw new MasterNotRunningException(e); - } - } - resetMasterServiceState(this.masterServiceState); - } - // Ugly delegation just so we can add in a Close method. - final MasterService.BlockingInterface stub = this.masterServiceState.stub; - return new MasterKeepAliveConnection() { - MasterServiceState mss = masterServiceState; - @Override - public AddColumnResponse addColumn(RpcController controller, AddColumnRequest request) - throws ServiceException { - return stub.addColumn(controller, request); - } - - @Override - public DeleteColumnResponse deleteColumn(RpcController controller, - DeleteColumnRequest request) - throws ServiceException { - return stub.deleteColumn(controller, request); - } - - @Override - public ModifyColumnResponse modifyColumn(RpcController controller, - ModifyColumnRequest request) - throws ServiceException { - return stub.modifyColumn(controller, request); - } - - @Override - public MoveRegionResponse moveRegion(RpcController controller, - MoveRegionRequest request) throws ServiceException { - return stub.moveRegion(controller, request); - } - - @Override - public DispatchMergingRegionsResponse dispatchMergingRegions( - RpcController controller, DispatchMergingRegionsRequest request) - throws ServiceException { - return stub.dispatchMergingRegions(controller, request); - } - - @Override - public AssignRegionResponse assignRegion(RpcController controller, - AssignRegionRequest request) throws ServiceException { - return stub.assignRegion(controller, request); - } - - @Override - public UnassignRegionResponse unassignRegion(RpcController controller, - UnassignRegionRequest request) throws ServiceException { - return stub.unassignRegion(controller, request); - } - - @Override - public OfflineRegionResponse offlineRegion(RpcController controller, - OfflineRegionRequest request) throws ServiceException { - return stub.offlineRegion(controller, request); - } - - @Override - public DeleteTableResponse deleteTable(RpcController controller, - DeleteTableRequest request) throws ServiceException { - return stub.deleteTable(controller, request); - } - - @Override - public TruncateTableResponse truncateTable(RpcController controller, - TruncateTableRequest request) throws ServiceException { - return stub.truncateTable(controller, request); - } - - @Override - public EnableTableResponse enableTable(RpcController controller, - EnableTableRequest request) throws ServiceException { - return stub.enableTable(controller, request); - } - - @Override - public DisableTableResponse disableTable(RpcController controller, - DisableTableRequest request) throws ServiceException { - return stub.disableTable(controller, request); - } - - @Override - public ModifyTableResponse modifyTable(RpcController controller, - ModifyTableRequest request) throws ServiceException { - return stub.modifyTable(controller, request); - } - - @Override - public CreateTableResponse createTable(RpcController controller, - CreateTableRequest request) throws ServiceException { - return stub.createTable(controller, request); - } - - @Override - public ShutdownResponse shutdown(RpcController controller, - ShutdownRequest request) throws ServiceException { - return stub.shutdown(controller, request); - } - - @Override - public StopMasterResponse stopMaster(RpcController controller, - StopMasterRequest request) throws ServiceException { - return stub.stopMaster(controller, request); - } - - @Override - public BalanceResponse balance(RpcController controller, - BalanceRequest request) throws ServiceException { - return stub.balance(controller, request); - } - - @Override - public SetBalancerRunningResponse setBalancerRunning( - RpcController controller, SetBalancerRunningRequest request) - throws ServiceException { - return stub.setBalancerRunning(controller, request); - } - - @Override - public RunCatalogScanResponse runCatalogScan(RpcController controller, - RunCatalogScanRequest request) throws ServiceException { - return stub.runCatalogScan(controller, request); - } - - @Override - public EnableCatalogJanitorResponse enableCatalogJanitor( - RpcController controller, EnableCatalogJanitorRequest request) - throws ServiceException { - return stub.enableCatalogJanitor(controller, request); - } - - @Override - public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled( - RpcController controller, IsCatalogJanitorEnabledRequest request) - throws ServiceException { - return stub.isCatalogJanitorEnabled(controller, request); - } - - @Override - public CoprocessorServiceResponse execMasterService( - RpcController controller, CoprocessorServiceRequest request) - throws ServiceException { - return stub.execMasterService(controller, request); - } - - @Override - public SnapshotResponse snapshot(RpcController controller, - SnapshotRequest request) throws ServiceException { - return stub.snapshot(controller, request); - } - - @Override - public GetCompletedSnapshotsResponse getCompletedSnapshots( - RpcController controller, GetCompletedSnapshotsRequest request) - throws ServiceException { - return stub.getCompletedSnapshots(controller, request); - } - - @Override - public DeleteSnapshotResponse deleteSnapshot(RpcController controller, - DeleteSnapshotRequest request) throws ServiceException { - return stub.deleteSnapshot(controller, request); - } - - @Override - public IsSnapshotDoneResponse isSnapshotDone(RpcController controller, - IsSnapshotDoneRequest request) throws ServiceException { - return stub.isSnapshotDone(controller, request); - } - - @Override - public RestoreSnapshotResponse restoreSnapshot( - RpcController controller, RestoreSnapshotRequest request) - throws ServiceException { - return stub.restoreSnapshot(controller, request); - } - - @Override - public IsRestoreSnapshotDoneResponse isRestoreSnapshotDone( - RpcController controller, IsRestoreSnapshotDoneRequest request) - throws ServiceException { - return stub.isRestoreSnapshotDone(controller, request); - } - - @Override - public ExecProcedureResponse execProcedure( - RpcController controller, ExecProcedureRequest request) - throws ServiceException { - return stub.execProcedure(controller, request); - } - - @Override - public ExecProcedureResponse execProcedureWithRet( - RpcController controller, ExecProcedureRequest request) - throws ServiceException { - return stub.execProcedureWithRet(controller, request); - } - - @Override - public IsProcedureDoneResponse isProcedureDone(RpcController controller, - IsProcedureDoneRequest request) throws ServiceException { - return stub.isProcedureDone(controller, request); - } - - @Override - public IsMasterRunningResponse isMasterRunning( - RpcController controller, IsMasterRunningRequest request) - throws ServiceException { - return stub.isMasterRunning(controller, request); - } - - @Override - public ModifyNamespaceResponse modifyNamespace(RpcController controller, - ModifyNamespaceRequest request) - throws ServiceException { - return stub.modifyNamespace(controller, request); - } - - @Override - public CreateNamespaceResponse createNamespace( - RpcController controller, CreateNamespaceRequest request) throws ServiceException { - return stub.createNamespace(controller, request); - } - - @Override - public DeleteNamespaceResponse deleteNamespace( - RpcController controller, DeleteNamespaceRequest request) throws ServiceException { - return stub.deleteNamespace(controller, request); - } - - @Override - public GetNamespaceDescriptorResponse getNamespaceDescriptor(RpcController controller, - GetNamespaceDescriptorRequest request) throws ServiceException { - return stub.getNamespaceDescriptor(controller, request); - } - - @Override - public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller, - ListNamespaceDescriptorsRequest request) throws ServiceException { - return stub.listNamespaceDescriptors(controller, request); - } - - @Override - public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace( - RpcController controller, ListTableDescriptorsByNamespaceRequest request) - throws ServiceException { - return stub.listTableDescriptorsByNamespace(controller, request); - } - - @Override - public ListTableNamesByNamespaceResponse listTableNamesByNamespace( - RpcController controller, ListTableNamesByNamespaceRequest request) - throws ServiceException { - return stub.listTableNamesByNamespace(controller, request); - } - - @Override - public GetTableStateResponse getTableState( - RpcController controller, GetTableStateRequest request) - throws ServiceException { - return stub.getTableState(controller, request); - } - - @Override - public void close() { - release(this.mss); - } - - @Override - public GetSchemaAlterStatusResponse getSchemaAlterStatus( - RpcController controller, GetSchemaAlterStatusRequest request) - throws ServiceException { - return stub.getSchemaAlterStatus(controller, request); - } - - @Override - public GetTableDescriptorsResponse getTableDescriptors( - RpcController controller, GetTableDescriptorsRequest request) - throws ServiceException { - return stub.getTableDescriptors(controller, request); - } - - @Override - public GetTableNamesResponse getTableNames( - RpcController controller, GetTableNamesRequest request) - throws ServiceException { - return stub.getTableNames(controller, request); - } - - @Override - public GetClusterStatusResponse getClusterStatus( - RpcController controller, GetClusterStatusRequest request) - throws ServiceException { - return stub.getClusterStatus(controller, request); - } - - @Override - public SetQuotaResponse setQuota( - RpcController controller, SetQuotaRequest request) - throws ServiceException { - return stub.setQuota(controller, request); - } - - @Override - public MajorCompactionTimestampResponse getLastMajorCompactionTimestamp( - RpcController controller, MajorCompactionTimestampRequest request) - throws ServiceException { - return stub.getLastMajorCompactionTimestamp(controller, request); - } - - @Override - public MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion( - RpcController controller, MajorCompactionTimestampForRegionRequest request) - throws ServiceException { - return stub.getLastMajorCompactionTimestampForRegion(controller, request); - } - }; - } - - - private static void release(MasterServiceState mss) { - if (mss != null && mss.connection != null) { - ((HConnectionImplementation)mss.connection).releaseMaster(mss); - } - } - - private boolean isKeepAliveMasterConnectedAndRunning(MasterServiceState mss) { - if (mss.getStub() == null){ - return false; - } - try { - return mss.isMasterRunning(); - } catch (UndeclaredThrowableException e) { - // It's somehow messy, but we can receive exceptions such as - // java.net.ConnectException but they're not declared. So we catch it... - LOG.info("Master connection is not running anymore", e.getUndeclaredThrowable()); - return false; - } catch (ServiceException se) { - LOG.warn("Checking master connection", se); - return false; - } - } - - void releaseMaster(MasterServiceState mss) { - if (mss.getStub() == null) return; - synchronized (masterAndZKLock) { - --mss.userCount; - } - } - - private void closeMasterService(MasterServiceState mss) { - if (mss.getStub() != null) { - LOG.info("Closing master protocol: " + mss); - mss.clearStub(); - } - mss.userCount = 0; - } - - /** - * Immediate close of the shared master. Can be by the delayed close or when closing the - * connection itself. - */ - private void closeMaster() { - synchronized (masterAndZKLock) { - closeMasterService(masterServiceState); - } - } - - void updateCachedLocation(HRegionInfo hri, ServerName source, - ServerName serverName, long seqNum) { - HRegionLocation newHrl = new HRegionLocation(hri, serverName, seqNum); - cacheLocation(hri.getTable(), source, newHrl); - } - - @Override - public void deleteCachedRegionLocation(final HRegionLocation location) { - metaCache.clearCache(location); - } - - @Override - public void updateCachedLocations(final TableName tableName, byte[] rowkey, - final Object exception, final HRegionLocation source) { - assert source != null; - updateCachedLocations(tableName, source.getRegionInfo().getRegionName() - , rowkey, exception, source.getServerName()); - } - - /** - * Update the location with the new value (if the exception is a RegionMovedException) - * or delete it from the cache. Does nothing if we can be sure from the exception that - * the location is still accurate, or if the cache has already been updated. - * @param exception an object (to simplify user code) on which we will try to find a nested - * or wrapped or both RegionMovedException - * @param source server that is the source of the location update. - */ - @Override - public void updateCachedLocations(final TableName tableName, byte[] regionName, byte[] rowkey, - final Object exception, final ServerName source) { - if (rowkey == null || tableName == null) { - LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) + - ", tableName=" + (tableName == null ? "null" : tableName)); - return; - } - - if (source == null) { - // This should not happen, but let's secure ourselves. - return; - } - - if (regionName == null) { - // we do not know which region, so just remove the cache entry for the row and server - metaCache.clearCache(tableName, rowkey, source); - return; - } - - // Is it something we have already updated? - final RegionLocations oldLocations = getCachedLocation(tableName, rowkey); - HRegionLocation oldLocation = null; - if (oldLocations != null) { - oldLocation = oldLocations.getRegionLocationByRegionName(regionName); - } - if (oldLocation == null || !source.equals(oldLocation.getServerName())) { - // There is no such location in the cache (it's been removed already) or - // the cache has already been refreshed with a different location. => nothing to do - return; - } - - HRegionInfo regionInfo = oldLocation.getRegionInfo(); - Throwable cause = findException(exception); - if (cause != null) { - if (cause instanceof RegionTooBusyException || cause instanceof RegionOpeningException) { - // We know that the region is still on this region server - return; - } - - if (cause instanceof RegionMovedException) { - RegionMovedException rme = (RegionMovedException) cause; - if (LOG.isTraceEnabled()) { - LOG.trace("Region " + regionInfo.getRegionNameAsString() + " moved to " + - rme.getHostname() + ":" + rme.getPort() + - " according to " + source.getHostAndPort()); - } - // We know that the region is not anymore on this region server, but we know - // the new location. - updateCachedLocation( - regionInfo, source, rme.getServerName(), rme.getLocationSeqNum()); - return; - } - } - - // If we're here, it means that can cannot be sure about the location, so we remove it from - // the cache. Do not send the source because source can be a new server in the same host:port - metaCache.clearCache(regionInfo); - } - - @Override - public void updateCachedLocations(final byte[] tableName, byte[] rowkey, - final Object exception, final HRegionLocation source) { - updateCachedLocations(TableName.valueOf(tableName), rowkey, exception, source); - } - - /** - * @deprecated since 0.96 - Use {@link HTableInterface#batch} instead - */ - @Override - @Deprecated - public void processBatch(List list, - final TableName tableName, - ExecutorService pool, - Object[] results) throws IOException, InterruptedException { - // This belongs in HTable!!! Not in here. St.Ack - - // results must be the same size as list - if (results.length != list.size()) { - throw new IllegalArgumentException( - "argument results must be the same size as argument list"); - } - processBatchCallback(list, tableName, pool, results, null); - } - - /** - * @deprecated Unsupported API - */ - @Override - @Deprecated - public void processBatch(List list, - final byte[] tableName, - ExecutorService pool, - Object[] results) throws IOException, InterruptedException { - processBatch(list, TableName.valueOf(tableName), pool, results); - } - - /** - * Send the queries in parallel on the different region servers. Retries on failures. - * If the method returns it means that there is no error, and the 'results' array will - * contain no exception. On error, an exception is thrown, and the 'results' array will - * contain results and exceptions. - * @deprecated since 0.96 - Use {@link HTable#processBatchCallback} instead - */ - @Override - @Deprecated - public void processBatchCallback( - List list, - TableName tableName, - ExecutorService pool, - Object[] results, - Batch.Callback callback) - throws IOException, InterruptedException { - - AsyncRequestFuture ars = this.asyncProcess.submitAll( - pool, tableName, list, callback, results); - ars.waitUntilDone(); - if (ars.hasError()) { - throw ars.getErrors(); - } - } - - /** - * @deprecated Unsupported API - */ - @Override - @Deprecated - public void processBatchCallback( - List list, - byte[] tableName, - ExecutorService pool, - Object[] results, - Batch.Callback callback) - throws IOException, InterruptedException { - processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback); - } - - // For tests to override. - protected AsyncProcess createAsyncProcess(Configuration conf) { - // No default pool available. - return new AsyncProcess(this, conf, this.batchPool, - RpcRetryingCallerFactory.instantiate(conf, this.getStatisticsTracker()), false, - RpcControllerFactory.instantiate(conf)); - } - - @Override - public AsyncProcess getAsyncProcess() { - return asyncProcess; - } - - @Override - public ServerStatisticTracker getStatisticsTracker() { - return this.stats; - } - - @Override - public ClientBackoffPolicy getBackoffPolicy() { - return this.backoffPolicy; - } - - /* - * Return the number of cached region for a table. It will only be called - * from a unit test. - */ - @VisibleForTesting - int getNumberOfCachedRegionLocations(final TableName tableName) { - return metaCache.getNumberOfCachedRegionLocations(tableName); - } - - /** - * @deprecated always return false since 0.99 - */ - @Override - @Deprecated - public void setRegionCachePrefetch(final TableName tableName, final boolean enable) { - } - - /** - * @deprecated always return false since 0.99 - */ - @Override - @Deprecated - public void setRegionCachePrefetch(final byte[] tableName, - final boolean enable) { - } - - /** - * @deprecated always return false since 0.99 - */ - @Override - @Deprecated - public boolean getRegionCachePrefetch(TableName tableName) { - return false; - } - - /** - * @deprecated always return false since 0.99 - */ - @Override - @Deprecated - public boolean getRegionCachePrefetch(byte[] tableName) { - return false; - } - - @Override - public void abort(final String msg, Throwable t) { - if (t instanceof KeeperException.SessionExpiredException - && keepAliveZookeeper != null) { - synchronized (masterAndZKLock) { - if (keepAliveZookeeper != null) { - LOG.warn("This client just lost it's session with ZooKeeper," + - " closing it." + - " It will be recreated next time someone needs it", t); - closeZooKeeperWatcher(); - } - } - } else { - if (t != null) { - LOG.fatal(msg, t); - } else { - LOG.fatal(msg); - } - this.aborted = true; - close(); - this.closed = true; - } - } - - @Override - public boolean isClosed() { - return this.closed; - } - - @Override - public boolean isAborted(){ - return this.aborted; - } - - @Override - public int getCurrentNrHRS() throws IOException { - return this.registry.getCurrentNrHRS(); - } - - /** - * Increment this client's reference count. - */ - void incCount() { - ++refCount; - } - - /** - * Decrement this client's reference count. - */ - void decCount() { - if (refCount > 0) { - --refCount; - } - } - - /** - * Return if this client has no reference - * - * @return true if this client has no reference; false otherwise - */ - boolean isZeroReference() { - return refCount == 0; - } - - void internalClose() { - if (this.closed) { - return; - } - closeMaster(); - shutdownPools(); - this.closed = true; - closeZooKeeperWatcher(); - this.stubs.clear(); - if (clusterStatusListener != null) { - clusterStatusListener.close(); - } - if (rpcClient != null) { - rpcClient.close(); - } - } - - @Override - public void close() { - if (managed) { - if (aborted) { - ConnectionManager.deleteStaleConnection(this); - } else { - ConnectionManager.deleteConnection(this, false); - } - } else { - internalClose(); - } - } - - /** - * Close the connection for good, regardless of what the current value of - * {@link #refCount} is. Ideally, {@link #refCount} should be zero at this - * point, which would be the case if all of its consumers close the - * connection. However, on the off chance that someone is unable to close - * the connection, perhaps because it bailed out prematurely, the method - * below will ensure that this {@link HConnection} instance is cleaned up. - * Caveat: The JVM may take an unknown amount of time to call finalize on an - * unreachable object, so our hope is that every consumer cleans up after - * itself, like any good citizen. - */ - @Override - protected void finalize() throws Throwable { - super.finalize(); - // Pretend as if we are about to release the last remaining reference - refCount = 1; - close(); - } - - /** - * @deprecated Use {@link Admin#listTables()} instead - */ - @Deprecated - @Override - public HTableDescriptor[] listTables() throws IOException { - MasterKeepAliveConnection master = getKeepAliveMasterService(); - try { - GetTableDescriptorsRequest req = - RequestConverter.buildGetTableDescriptorsRequest((List)null); - return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req)); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } finally { - master.close(); - } - } - - /** - * @deprecated Use {@link Admin#listTableNames()} instead - */ - @Deprecated - @Override - public String[] getTableNames() throws IOException { - TableName[] tableNames = listTableNames(); - String[] result = new String[tableNames.length]; - for (int i = 0; i < tableNames.length; i++) { - result[i] = tableNames[i].getNameAsString(); - } - return result; - } - - /** - * @deprecated Use {@link Admin#listTableNames()} instead - */ - @Deprecated - @Override - public TableName[] listTableNames() throws IOException { - MasterKeepAliveConnection master = getKeepAliveMasterService(); - try { - return ProtobufUtil.getTableNameArray(master.getTableNames(null, - GetTableNamesRequest.newBuilder().build()) - .getTableNamesList()); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } finally { - master.close(); - } - } - - /** - * @deprecated Use {@link Admin#getTableDescriptorsByTableName(List)} instead - */ - @Deprecated - @Override - public HTableDescriptor[] getHTableDescriptorsByTableName( - List tableNames) throws IOException { - if (tableNames == null || tableNames.isEmpty()) return new HTableDescriptor[0]; - MasterKeepAliveConnection master = getKeepAliveMasterService(); - try { - GetTableDescriptorsRequest req = - RequestConverter.buildGetTableDescriptorsRequest(tableNames); - return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req)); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } finally { - master.close(); - } - } - - /** - * @deprecated Use {@link Admin#getTableDescriptorsByTableName(List)} instead - */ - @Deprecated - @Override - public HTableDescriptor[] getHTableDescriptors( - List names) throws IOException { - List tableNames = new ArrayList(names.size()); - for(String name : names) { - tableNames.add(TableName.valueOf(name)); - } - - return getHTableDescriptorsByTableName(tableNames); - } - - @Override - public NonceGenerator getNonceGenerator() { - return this.nonceGenerator; - } - - /** - * Connects to the master to get the table descriptor. - * @param tableName table name - * @throws IOException if the connection to master fails or if the table - * is not found. - * @deprecated Use {@link Admin#getTableDescriptor(TableName)} instead - */ - @Deprecated - @Override - public HTableDescriptor getHTableDescriptor(final TableName tableName) - throws IOException { - if (tableName == null) return null; - MasterKeepAliveConnection master = getKeepAliveMasterService(); - GetTableDescriptorsResponse htds; - try { - GetTableDescriptorsRequest req = - RequestConverter.buildGetTableDescriptorsRequest(tableName); - htds = master.getTableDescriptors(null, req); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } finally { - master.close(); - } - if (!htds.getTableSchemaList().isEmpty()) { - return HTableDescriptor.convert(htds.getTableSchemaList().get(0)); - } - throw new TableNotFoundException(tableName.getNameAsString()); - } - - /** - * @deprecated Use {@link Admin#getTableDescriptor(TableName)} instead - */ - @Deprecated - @Override - public HTableDescriptor getHTableDescriptor(final byte[] tableName) - throws IOException { - return getHTableDescriptor(TableName.valueOf(tableName)); - } - - @Override - public TableState getTableState(TableName tableName) throws IOException { - ClusterConnection conn = getConnectionInternal(getConfiguration()); - TableState tableState = MetaTableAccessor.getTableState(conn, tableName); - if (tableState == null) - throw new TableNotFoundException(tableName); - return tableState; - } - - @Override - public RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf) { - return RpcRetryingCallerFactory - .instantiate(conf, this.interceptor, this.getStatisticsTracker()); - } - - @Override - public boolean isManaged() { - return managed; - } - } - /** * The record of errors for servers. */ diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java index a035e2f..156ec30 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java @@ -82,7 +82,7 @@ public final class ConnectionUtils { */ public static NonceGenerator injectNonceGeneratorForTesting( ClusterConnection conn, NonceGenerator cnm) { - return ConnectionManager.injectNonceGeneratorForTesting(conn, cnm); + return HConnectionImplementation.injectNonceGeneratorForTesting(conn, cnm); } /** @@ -146,7 +146,7 @@ public final class ConnectionUtils { * Some tests shut down the master. But table availability is a master RPC which is performed on * region re-lookups. */ - static class MasterlessConnection extends ConnectionManager.HConnectionImplementation { + static class MasterlessConnection extends HConnectionImplementation { MasterlessConnection(Configuration conf, boolean managed, ExecutorService pool, User user) throws IOException { super(conf, managed, pool, user); diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index f372528..37e3709 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -2561,7 +2561,7 @@ public class HBaseAdmin implements Admin { ZooKeeperKeepAliveConnection zkw = null; try { // This is NASTY. FIX!!!! Dependent on internal implementation! TODO - zkw = ((ConnectionManager.HConnectionImplementation)connection). + zkw = ((HConnectionImplementation)connection). getKeepAliveZooKeeperWatcher(); zkw.getRecoverableZooKeeper().getZooKeeper().exists(zkw.baseZNode, false); } catch (IOException e) { @@ -3781,8 +3781,8 @@ public class HBaseAdmin implements Admin { @Override public int getMasterInfoPort() throws IOException { // TODO: Fix! Reaching into internal implementation!!!! - ConnectionManager.HConnectionImplementation connection = - (ConnectionManager.HConnectionImplementation)this.connection; + HConnectionImplementation connection = + (HConnectionImplementation)this.connection; ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher(); try { return MasterAddressTracker.getMasterInfoPort(zkw); 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/HConnectionImplementation.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionImplementation.java new file mode 100644 index 0000000..73a0f1c --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionImplementation.java @@ -0,0 +1,2132 @@ +/** + * + * 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 com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.BlockingRpcChannel; +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.RegionTooBusyException; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.TableNotEnabledException; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; +import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory; +import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.exceptions.RegionMovedException; +import org.apache.hadoop.hbase.exceptions.RegionOpeningException; +import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.RequestConverter; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.protobuf.generated.MasterProtos; +import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ExceptionUtil; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.zookeeper.KeeperException; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.lang.reflect.UndeclaredThrowableException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** Encapsulates connection to zookeeper and regionservers.*/ +@edu.umd.cs.findbugs.annotations.SuppressWarnings( + value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION", + justification="Access to the conncurrent hash map is under a lock so should be fine.") +class HConnectionImplementation implements ClusterConnection, Closeable { + static final Log LOG = LogFactory.getLog(HConnectionImplementation.class); + private static final String CLIENT_NONCES_ENABLED_KEY = "hbase.client.nonces.enabled"; + + private final long pause; + private final boolean useMetaReplicas; + private final int numTries; + final int rpcTimeout; + + /** + * Global nonceGenerator shared per client.Currently there's no reason to limit its scope. + * Once it's set under nonceGeneratorCreateLock, it is never unset or changed. + */ + private static volatile NonceGenerator nonceGenerator = null; + /** The nonce generator lock. Only taken when creating HConnection, which gets a private copy. */ + private static Object nonceGeneratorCreateLock = new Object(); + + private final AsyncProcess asyncProcess; + // single tracker per connection + private final ServerStatisticTracker stats; + + private volatile boolean closed; + private volatile boolean aborted; + + // package protected for the tests + ClusterStatusListener clusterStatusListener; + + + private final Object metaRegionLock = new Object(); + + // We have a single lock for master & zk to prevent deadlocks. Having + // one lock for ZK and one lock for master is not possible: + // When creating a connection to master, we need a connection to ZK to get + // its address. But another thread could have taken the ZK lock, and could + // be waiting for the master lock => deadlock. + private final Object masterAndZKLock = new Object(); + + private long keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE; + + // thread executor shared by all HTableInterface instances created + // by this connection + private volatile ExecutorService batchPool = null; + // meta thread executor shared by all HTableInterface instances created + // by this connection + private volatile ExecutorService metaLookupPool = null; + private volatile boolean cleanupPool = false; + + private final Configuration conf; + + // cache the configuration value for tables so that we can avoid calling + // the expensive Configuration to fetch the value multiple times. + private final TableConfiguration tableConfig; + + // Client rpc instance. + private RpcClient rpcClient; + + private MetaCache metaCache = new MetaCache(); + + private int refCount; + + // indicates whether this connection's life cycle is managed (by us) + private boolean managed; + + private User user; + + private RpcRetryingCallerFactory rpcCallerFactory; + + private RpcControllerFactory rpcControllerFactory; + + private final RetryingCallerInterceptor interceptor; + + /** + * Cluster registry of basic info such as clusterid and meta region location. + */ + Registry registry; + + private final ClientBackoffPolicy backoffPolicy; + + HConnectionImplementation(Configuration conf, boolean managed) throws IOException { + this(conf, managed, null, null); + } + + /** + * constructor + * @param conf Configuration object + * @param managed If true, does not do full shutdown on close; i.e. cleanup of connection + * to zk and shutdown of all services; we just close down the resources this connection was + * responsible for and decrement usage counters. It is up to the caller to do the full + * cleanup. It is set when we want have connection sharing going on -- reuse of zk connection, + * and cached region locations, established regionserver connections, etc. When connections + * are shared, we have reference counting going on and will only do full cleanup when no more + * users of an HConnectionImplementation instance. + */ + HConnectionImplementation(Configuration conf, boolean managed, + ExecutorService pool, User user) throws IOException { + this(conf); + this.user = user; + this.batchPool = pool; + this.managed = managed; + this.registry = setupRegistry(); + retrieveClusterId(); + + this.rpcClient = RpcClientFactory.createClient(this.conf, this.clusterId); + this.rpcControllerFactory = RpcControllerFactory.instantiate(conf); + + // Do we publish the status? + boolean shouldListen = conf.getBoolean(HConstants.STATUS_PUBLISHED, + HConstants.STATUS_PUBLISHED_DEFAULT); + Class listenerClass = + conf.getClass(ClusterStatusListener.STATUS_LISTENER_CLASS, + ClusterStatusListener.DEFAULT_STATUS_LISTENER_CLASS, + ClusterStatusListener.Listener.class); + if (shouldListen) { + if (listenerClass == null) { + LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " + + ClusterStatusListener.STATUS_LISTENER_CLASS + " is not set - not listening status"); + } else { + clusterStatusListener = new ClusterStatusListener( + new ClusterStatusListener.DeadServerHandler() { + @Override + public void newDead(ServerName sn) { + clearCaches(sn); + rpcClient.cancelConnections(sn); + } + }, conf, listenerClass); + } + } + } + + /** + * For tests. + */ + protected HConnectionImplementation(Configuration conf) { + this.conf = conf; + this.tableConfig = new TableConfiguration(conf); + this.closed = false; + this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, + HConstants.DEFAULT_HBASE_CLIENT_PAUSE); + this.useMetaReplicas = conf.getBoolean(HConstants.USE_META_REPLICAS, + HConstants.DEFAULT_USE_META_REPLICAS); + this.numTries = tableConfig.getRetriesNumber(); + this.rpcTimeout = conf.getInt( + HConstants.HBASE_RPC_TIMEOUT_KEY, + HConstants.DEFAULT_HBASE_RPC_TIMEOUT); + if (conf.getBoolean(CLIENT_NONCES_ENABLED_KEY, true)) { + synchronized (nonceGeneratorCreateLock) { + if (nonceGenerator == null) { + nonceGenerator = new PerClientRandomNonceGenerator(); + } + } + } else { + nonceGenerator = new ConnectionManager.NoNonceGenerator(); + } + stats = ServerStatisticTracker.create(conf); + this.asyncProcess = createAsyncProcess(this.conf); + this.interceptor = (new RetryingCallerInterceptorFactory(conf)).build(); + this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, interceptor, this.stats); + this.backoffPolicy = ClientBackoffPolicyFactory.create(conf); + } + + /** + * @param conn The connection for which to replace the generator. + * @param cnm Replaces the nonce generator used, for testing. + * @return old nonce generator. + */ + @VisibleForTesting + static NonceGenerator injectNonceGeneratorForTesting( + ClusterConnection conn, NonceGenerator cnm) { + HConnectionImplementation connImpl = (HConnectionImplementation)conn; + NonceGenerator ng = connImpl.getNonceGenerator(); + ConnectionManager.LOG.warn("Nonce generator is being replaced by test code for " + + cnm.getClass().getName()); + nonceGenerator = cnm; + return ng; + } + + @Override + public HTableInterface getTable(String tableName) throws IOException { + return getTable(TableName.valueOf(tableName)); + } + + @Override + public HTableInterface getTable(byte[] tableName) throws IOException { + return getTable(TableName.valueOf(tableName)); + } + + @Override + public HTableInterface getTable(TableName tableName) throws IOException { + return getTable(tableName, getBatchPool()); + } + + @Override + public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException { + return getTable(TableName.valueOf(tableName), pool); + } + + @Override + public HTableInterface getTable(byte[] tableName, ExecutorService pool) throws IOException { + return getTable(TableName.valueOf(tableName), pool); + } + + @Override + public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException { + if (managed) { + throw new NeedUnmanagedConnectionException(); + } + return new HTable(tableName, this, tableConfig, rpcCallerFactory, rpcControllerFactory, pool); + } + + @Override + public BufferedMutator getBufferedMutator(BufferedMutatorParams params) { + if (params.getTableName() == null) { + throw new IllegalArgumentException("TableName cannot be null."); + } + if (params.getPool() == null) { + params.pool(HTable.getDefaultExecutor(getConfiguration())); + } + if (params.getWriteBufferSize() == BufferedMutatorParams.UNSET) { + params.writeBufferSize(tableConfig.getWriteBufferSize()); + } + if (params.getMaxKeyValueSize() == BufferedMutatorParams.UNSET) { + params.maxKeyValueSize(tableConfig.getMaxKeyValueSize()); + } + return new BufferedMutatorImpl(this, rpcCallerFactory, rpcControllerFactory, params); + } + + @Override + public BufferedMutator getBufferedMutator(TableName tableName) { + return getBufferedMutator(new BufferedMutatorParams(tableName)); + } + + @Override + public RegionLocator getRegionLocator(TableName tableName) throws IOException { + return new HRegionLocator(tableName, this); + } + + @Override + public Admin getAdmin() throws IOException { + if (managed) { + throw new NeedUnmanagedConnectionException(); + } + return new HBaseAdmin(this); + } + + private ExecutorService getBatchPool() { + if (batchPool == null) { + synchronized (this) { + if (batchPool == null) { + this.batchPool = getThreadPool(conf.getInt("hbase.hconnection.threads.max", 256), + conf.getInt("hbase.hconnection.threads.core", 256), "-shared-"); + this.cleanupPool = true; + } + } + } + return this.batchPool; + } + + private ExecutorService getThreadPool(int maxThreads, int coreThreads, String nameHint) { + // shared HTable thread executor not yet initialized + if (maxThreads == 0) { + maxThreads = Runtime.getRuntime().availableProcessors() * 8; + } + if (coreThreads == 0) { + coreThreads = Runtime.getRuntime().availableProcessors() * 8; + } + long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 60); + LinkedBlockingQueue workQueue = + new LinkedBlockingQueue(maxThreads * + conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, + HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS)); + ThreadPoolExecutor tpe = new ThreadPoolExecutor( + coreThreads, + maxThreads, + keepAliveTime, + TimeUnit.SECONDS, + workQueue, + Threads.newDaemonThreadFactory(toString() + nameHint)); + tpe.allowCoreThreadTimeOut(true); + return tpe; + } + + private ExecutorService getMetaLookupPool() { + if (this.metaLookupPool == null) { + synchronized (this) { + if (this.metaLookupPool == null) { + //The meta lookup can happen on replicas of the meta (if the appropriate configs + //are enabled).In a replicated-meta setup, the number '3' is assumed as the max + //number of replicas by default (unless it is configured to be of a higher value). + //In a non-replicated-meta setup, only one thread would be active. + this.metaLookupPool = getThreadPool( + conf.getInt("hbase.hconnection.meta.lookup.threads.max", 3), + conf.getInt("hbase.hconnection.meta.lookup.threads.max.core", 3), + "-metaLookup-shared-"); + } + } + } + return this.metaLookupPool; + } + + protected ExecutorService getCurrentMetaLookupPool() { + return metaLookupPool; + } + + protected ExecutorService getCurrentBatchPool() { + return batchPool; + } + + private void shutdownPools() { + if (this.cleanupPool && this.batchPool != null && !this.batchPool.isShutdown()) { + shutdownBatchPool(this.batchPool); + } + if (this.metaLookupPool != null && !this.metaLookupPool.isShutdown()) { + shutdownBatchPool(this.metaLookupPool); + } + } + + private void shutdownBatchPool(ExecutorService pool) { + pool.shutdown(); + try { + if (!pool.awaitTermination(10, TimeUnit.SECONDS)) { + pool.shutdownNow(); + } + } catch (InterruptedException e) { + pool.shutdownNow(); + } + } + + /** + * @return The cluster registry implementation to use. + * @throws java.io.IOException + */ + private Registry setupRegistry() throws IOException { + return RegistryFactory.getRegistry(this); + } + + /** + * For tests only. + */ + @VisibleForTesting + RpcClient getRpcClient() { + return rpcClient; + } + + /** + * An identifier that will remain the same for a given connection. + */ + @Override + public String toString(){ + return "hconnection-0x" + Integer.toHexString(hashCode()); + } + + protected String clusterId = null; + + void retrieveClusterId() { + if (clusterId != null) return; + this.clusterId = this.registry.getClusterId(); + if (clusterId == null) { + clusterId = HConstants.CLUSTER_ID_DEFAULT; + LOG.debug("clusterid came back null, using default " + clusterId); + } + } + + @Override + public Configuration getConfiguration() { + return this.conf; + } + + private void checkIfBaseNodeAvailable(ZooKeeperWatcher zkw) + throws MasterNotRunningException { + String errorMsg; + try { + if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) { + errorMsg = "The node " + zkw.baseZNode+" is not in ZooKeeper. " + + "It should have been written by the master. " + + "Check the value configured in 'zookeeper.znode.parent'. " + + "There could be a mismatch with the one configured in the master."; + LOG.error(errorMsg); + throw new MasterNotRunningException(errorMsg); + } + } catch (KeeperException e) { + errorMsg = "Can't get connection to ZooKeeper: " + e.getMessage(); + LOG.error(errorMsg); + throw new MasterNotRunningException(errorMsg, e); + } + } + + /** + * @return true if the master is running, throws an exception otherwise + * @throws org.apache.hadoop.hbase.MasterNotRunningException - if the master is not running + * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException + * @deprecated this has been deprecated without a replacement + */ + @Deprecated + @Override + public boolean isMasterRunning() + throws MasterNotRunningException, ZooKeeperConnectionException { + // When getting the master connection, we check it's running, + // so if there is no exception, it means we've been able to get a + // connection on a running master + MasterKeepAliveConnection m = getKeepAliveMasterService(); + m.close(); + return true; + } + + @Override + public HRegionLocation getRegionLocation(final TableName tableName, + final byte [] row, boolean reload) + throws IOException { + return reload? relocateRegion(tableName, row): locateRegion(tableName, row); + } + + @Override + public HRegionLocation getRegionLocation(final byte[] tableName, + final byte [] row, boolean reload) + throws IOException { + return getRegionLocation(TableName.valueOf(tableName), row, reload); + } + + @Override + public boolean isTableEnabled(TableName tableName) throws IOException { + return getTableState(tableName).inStates(TableState.State.ENABLED); + } + + @Override + public boolean isTableEnabled(byte[] tableName) throws IOException { + return isTableEnabled(TableName.valueOf(tableName)); + } + + @Override + public boolean isTableDisabled(TableName tableName) throws IOException { + return getTableState(tableName).inStates(TableState.State.DISABLED); + } + + @Override + public boolean isTableDisabled(byte[] tableName) throws IOException { + return isTableDisabled(TableName.valueOf(tableName)); + } + + @Override + public boolean isTableAvailable(final TableName tableName) throws IOException { + return isTableAvailable(tableName, null); + } + + @Override + public boolean isTableAvailable(final byte[] tableName) throws IOException { + return isTableAvailable(TableName.valueOf(tableName)); + } + + @Override + public boolean isTableAvailable(final TableName tableName, @Nullable final byte[][] splitKeys) + throws IOException { + try { + if (!isTableEnabled(tableName)) { + LOG.debug("Table " + tableName + " not enabled"); + return false; + } + ClusterConnection connection = ConnectionManager.getConnectionInternal(getConfiguration()); + List> locations = MetaTableAccessor + .getTableRegionsAndLocations(connection, tableName, true); + int notDeployed = 0; + int regionCount = 0; + for (Pair pair : locations) { + HRegionInfo info = pair.getFirst(); + if (pair.getSecond() == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Table " + tableName + " has not deployed region " + pair.getFirst() + .getEncodedName()); + } + notDeployed++; + } else if (splitKeys != null + && !Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) { + for (byte[] splitKey : splitKeys) { + // Just check if the splitkey is available + if (Bytes.equals(info.getStartKey(), splitKey)) { + regionCount++; + break; + } + } + } else { + // Always empty start row should be counted + regionCount++; + } + } + if (notDeployed > 0) { + if (LOG.isDebugEnabled()) { + LOG.debug("Table " + tableName + " has " + notDeployed + " regions"); + } + return false; + } else if (splitKeys != null && regionCount != splitKeys.length + 1) { + if (LOG.isDebugEnabled()) { + LOG.debug("Table " + tableName + " expected to have " + (splitKeys.length + 1) + + " regions, but only " + regionCount + " available"); + } + return false; + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Table " + tableName + " should be available"); + } + return true; + } + } catch (TableNotFoundException tnfe) { + LOG.warn("Table " + tableName + " not enabled, it is not exists"); + return false; + } + } + + @Override + public boolean isTableAvailable(final byte[] tableName, final byte[][] splitKeys) + throws IOException { + return isTableAvailable(TableName.valueOf(tableName), splitKeys); + } + + @Override + public HRegionLocation locateRegion(final byte[] regionName) throws IOException { + RegionLocations locations = locateRegion(HRegionInfo.getTable(regionName), + HRegionInfo.getStartKey(regionName), false, true); + return locations == null ? null : locations.getRegionLocation(); + } + + @Override + public boolean isDeadServer(ServerName sn) { + if (clusterStatusListener == null) { + return false; + } else { + return clusterStatusListener.isDeadServer(sn); + } + } + + @Override + public List locateRegions(final TableName tableName) + throws IOException { + return locateRegions(tableName, false, true); + } + + @Override + public List locateRegions(final byte[] tableName) + throws IOException { + return locateRegions(TableName.valueOf(tableName)); + } + + @Override + public List locateRegions(final TableName tableName, + final boolean useCache, final boolean offlined) throws IOException { + List regions = MetaTableAccessor + .getTableRegions(this, tableName, !offlined); + final List locations = new ArrayList(); + for (HRegionInfo regionInfo : regions) { + RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true); + if (list != null) { + for (HRegionLocation loc : list.getRegionLocations()) { + if (loc != null) { + locations.add(loc); + } + } + } + } + return locations; + } + + @Override + public List locateRegions(final byte[] tableName, + final boolean useCache, final boolean offlined) throws IOException { + return locateRegions(TableName.valueOf(tableName), useCache, offlined); + } + + @Override + public HRegionLocation locateRegion( + final TableName tableName, final byte[] row) throws IOException{ + RegionLocations locations = locateRegion(tableName, row, true, true); + return locations == null ? null : locations.getRegionLocation(); + } + + @Override + public HRegionLocation locateRegion(final byte[] tableName, + final byte [] row) + throws IOException{ + return locateRegion(TableName.valueOf(tableName), row); + } + + @Override + public HRegionLocation relocateRegion(final TableName tableName, + final byte [] row) throws IOException{ + RegionLocations locations = relocateRegion(tableName, row, + RegionReplicaUtil.DEFAULT_REPLICA_ID); + return locations == null ? null : + locations.getRegionLocation(RegionReplicaUtil.DEFAULT_REPLICA_ID); + } + + @Override + public RegionLocations relocateRegion(final TableName tableName, + final byte [] row, int replicaId) throws IOException{ + // Since this is an explicit request not to use any caching, finding + // disabled tables should not be desirable. This will ensure that an exception is thrown when + // the first time a disabled table is interacted with. + if (!tableName.equals(TableName.META_TABLE_NAME) && isTableDisabled(tableName)) { + throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled."); + } + + return locateRegion(tableName, row, false, true, replicaId); + } + + @Override + public HRegionLocation relocateRegion(final byte[] tableName, + final byte [] row) throws IOException { + return relocateRegion(TableName.valueOf(tableName), row); + } + + @Override + public RegionLocations locateRegion(final TableName tableName, + final byte [] row, boolean useCache, boolean retry) + throws IOException { + return locateRegion(tableName, row, useCache, retry, RegionReplicaUtil.DEFAULT_REPLICA_ID); + } + + @Override + public RegionLocations locateRegion(final TableName tableName, + final byte [] row, boolean useCache, boolean retry, int replicaId) + throws IOException { + if (this.closed) throw new IOException(toString() + " closed"); + if (tableName== null || tableName.getName().length == 0) { + throw new IllegalArgumentException( + "table name cannot be null or zero length"); + } + if (tableName.equals(TableName.META_TABLE_NAME)) { + return locateMeta(tableName, useCache, replicaId); + } else { + // Region not in the cache - have to go to the meta RS + return locateRegionInMeta(tableName, row, useCache, retry, replicaId); + } + } + + private RegionLocations locateMeta(final TableName tableName, + boolean useCache, int replicaId) throws IOException { + // HBASE-10785: We cache the location of the META itself, so that we are not overloading + // zookeeper with one request for every region lookup. We cache the META with empty row + // key in MetaCache. + byte[] metaCacheKey = HConstants.EMPTY_START_ROW; // use byte[0] as the row for meta + RegionLocations locations = null; + if (useCache) { + locations = getCachedLocation(tableName, metaCacheKey); + if (locations != null && locations.getRegionLocation(replicaId) != null) { + return locations; + } + } + + // only one thread should do the lookup. + synchronized (metaRegionLock) { + // Check the cache again for a hit in case some other thread made the + // same query while we were waiting on the lock. + if (useCache) { + locations = getCachedLocation(tableName, metaCacheKey); + if (locations != null && locations.getRegionLocation(replicaId) != null) { + return locations; + } + } + + // Look up from zookeeper + locations = this.registry.getMetaRegionLocation(); + if (locations != null) { + cacheLocation(tableName, locations); + } + } + return locations; + } + + /* + * Search the hbase:meta table for the HRegionLocation + * info that contains the table and row we're seeking. + */ + private RegionLocations locateRegionInMeta(TableName tableName, byte[] row, + boolean useCache, boolean retry, int replicaId) throws IOException { + + // If we are supposed to be using the cache, look in the cache to see if + // we already have the region. + if (useCache) { + RegionLocations locations = getCachedLocation(tableName, row); + if (locations != null && locations.getRegionLocation(replicaId) != null) { + return locations; + } + } + + // build the key of the meta region we should be looking for. + // the extra 9's on the end are necessary to allow "exact" matches + // without knowing the precise region names. + byte[] metaKey = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false); + + Scan s = new Scan(); + s.setReversed(true); + s.setStartRow(metaKey); + s.setSmall(true); + s.setCaching(1); + if (this.useMetaReplicas) { + s.setConsistency(Consistency.TIMELINE); + } + + int localNumRetries = (retry ? numTries : 1); + + for (int tries = 0; true; tries++) { + if (tries >= localNumRetries) { + throw new NoServerForRegionException("Unable to find region for " + + Bytes.toStringBinary(row) + " in " + tableName + + " after " + localNumRetries + " tries."); + } + if (useCache) { + RegionLocations locations = getCachedLocation(tableName, row); + if (locations != null && locations.getRegionLocation(replicaId) != null) { + return locations; + } + } else { + // If we are not supposed to be using the cache, delete any existing cached location + // so it won't interfere. + metaCache.clearCache(tableName, row); + } + + // Query the meta region + try { + Result regionInfoRow = null; + ReversedClientScanner rcs = null; + try { + rcs = new ClientSmallReversedScanner(conf, s, TableName.META_TABLE_NAME, this, + rpcCallerFactory, rpcControllerFactory, getMetaLookupPool(), 0); + regionInfoRow = rcs.next(); + } finally { + if (rcs != null) { + rcs.close(); + } + } + + if (regionInfoRow == null) { + throw new TableNotFoundException(tableName); + } + + // convert the row result into the HRegionLocation we need! + RegionLocations locations = MetaTableAccessor.getRegionLocations(regionInfoRow); + if (locations == null || locations.getRegionLocation(replicaId) == null) { + throw new IOException("HRegionInfo was null in " + + tableName + ", row=" + regionInfoRow); + } + HRegionInfo regionInfo = locations.getRegionLocation(replicaId).getRegionInfo(); + if (regionInfo == null) { + throw new IOException("HRegionInfo was null or empty in " + + TableName.META_TABLE_NAME + ", row=" + regionInfoRow); + } + + // possible we got a region of a different table... + if (!regionInfo.getTable().equals(tableName)) { + throw new TableNotFoundException( + "Table '" + tableName + "' was not found, got: " + + regionInfo.getTable() + "."); + } + if (regionInfo.isSplit()) { + throw new RegionOfflineException("the only available region for" + + " the required row is a split parent," + + " the daughters should be online soon: " + + regionInfo.getRegionNameAsString()); + } + if (regionInfo.isOffline()) { + throw new RegionOfflineException("the region is offline, could" + + " be caused by a disable table call: " + + regionInfo.getRegionNameAsString()); + } + + ServerName serverName = locations.getRegionLocation(replicaId).getServerName(); + if (serverName == null) { + throw new NoServerForRegionException("No server address listed " + + "in " + TableName.META_TABLE_NAME + " for region " + + regionInfo.getRegionNameAsString() + " containing row " + + Bytes.toStringBinary(row)); + } + + if (isDeadServer(serverName)){ + throw new RegionServerStoppedException("hbase:meta says the region "+ + regionInfo.getRegionNameAsString()+" is managed by the server " + serverName + + ", but it is dead."); + } + // Instantiate the location + cacheLocation(tableName, locations); + return locations; + } catch (TableNotFoundException e) { + // if we got this error, probably means the table just plain doesn't + // exist. rethrow the error immediately. this should always be coming + // from the HTable constructor. + throw e; + } catch (IOException e) { + ExceptionUtil.rethrowIfInterrupt(e); + + if (e instanceof RemoteException) { + e = ((RemoteException)e).unwrapRemoteException(); + } + if (tries < localNumRetries - 1) { + if (LOG.isDebugEnabled()) { + LOG.debug("locateRegionInMeta parentTable=" + + TableName.META_TABLE_NAME + ", metaLocation=" + + ", attempt=" + tries + " of " + + localNumRetries + " failed; retrying after sleep of " + + ConnectionUtils.getPauseTime(this.pause, tries) + " because: " + e.getMessage()); + } + } else { + throw e; + } + // Only relocate the parent region if necessary + if(!(e instanceof RegionOfflineException || + e instanceof NoServerForRegionException)) { + relocateRegion(TableName.META_TABLE_NAME, metaKey, replicaId); + } + } + try{ + Thread.sleep(ConnectionUtils.getPauseTime(this.pause, tries)); + } catch (InterruptedException e) { + throw new InterruptedIOException("Giving up trying to location region in " + + "meta: thread is interrupted."); + } + } + } + + /** + * Put a newly discovered HRegionLocation into the cache. + * @param tableName The table name. + * @param location the new location + */ + private void cacheLocation(final TableName tableName, final RegionLocations location) { + metaCache.cacheLocation(tableName, location); + } + + /** + * Search the cache for a location that fits our table and row key. + * Return null if no suitable region is located. + * + * @param tableName + * @param row + * @return Null or region location found in cache. + */ + RegionLocations getCachedLocation(final TableName tableName, + final byte [] row) { + return metaCache.getCachedLocation(tableName, row); + } + + public void clearRegionCache(final TableName tableName, byte[] row) { + metaCache.clearCache(tableName, row); + } + + /* + * Delete all cached entries of a table that maps to a specific location. + */ + @Override + public void clearCaches(final ServerName serverName) { + metaCache.clearCache(serverName); + } + + @Override + public void clearRegionCache() { + metaCache.clearCache(); + } + + @Override + public void clearRegionCache(final TableName tableName) { + metaCache.clearCache(tableName); + } + + @Override + public void clearRegionCache(final byte[] tableName) { + clearRegionCache(TableName.valueOf(tableName)); + } + + /** + * Put a newly discovered HRegionLocation into the cache. + * @param tableName The table name. + * @param source the source of the new location, if it's not coming from meta + * @param location the new location + */ + private void cacheLocation(final TableName tableName, final ServerName source, + final HRegionLocation location) { + metaCache.cacheLocation(tableName, source, location); + } + + // Map keyed by service name + regionserver to service stub implementation + private final ConcurrentHashMap stubs = + new ConcurrentHashMap(); + // Map of locks used creating service stubs per regionserver. + private final ConcurrentHashMap connectionLock = + new ConcurrentHashMap(); + + /** + * State of the MasterService connection/setup. + */ + static class MasterServiceState { + HConnection connection; + MasterProtos.MasterService.BlockingInterface stub; + int userCount; + + MasterServiceState(final HConnection connection) { + super(); + this.connection = connection; + } + + @Override + public String toString() { + return "MasterService"; + } + + Object getStub() { + return this.stub; + } + + void clearStub() { + this.stub = null; + } + + boolean isMasterRunning() throws ServiceException { + MasterProtos.IsMasterRunningResponse response = + this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest()); + return response != null? response.getIsMasterRunning(): false; + } + } + + /** + * Makes a client-side stub for master services. Sub-class to specialize. + * Depends on hosting class so not static. Exists so we avoid duplicating a bunch of code + * when setting up the MasterMonitorService and MasterAdminService. + */ + abstract class StubMaker { + /** + * Returns the name of the service stub being created. + */ + protected abstract String getServiceName(); + + /** + * Make stub and cache it internal so can be used later doing the isMasterRunning call. + * @param channel + */ + protected abstract Object makeStub(final BlockingRpcChannel channel); + + /** + * Once setup, check it works by doing isMasterRunning check. + * @throws com.google.protobuf.ServiceException + */ + protected abstract void isMasterRunning() throws ServiceException; + + /** + * Create a stub. Try once only. It is not typed because there is no common type to + * protobuf services nor their interfaces. Let the caller do appropriate casting. + * @return A stub for master services. + * @throws java.io.IOException + * @throws org.apache.zookeeper.KeeperException + * @throws com.google.protobuf.ServiceException + */ + private Object makeStubNoRetries() throws IOException, KeeperException, ServiceException { + ZooKeeperKeepAliveConnection zkw; + try { + zkw = getKeepAliveZooKeeperWatcher(); + } catch (IOException e) { + ExceptionUtil.rethrowIfInterrupt(e); + throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e); + } + try { + checkIfBaseNodeAvailable(zkw); + ServerName sn = MasterAddressTracker.getMasterAddress(zkw); + if (sn == null) { + String msg = "ZooKeeper available but no active master location found"; + LOG.info(msg); + throw new MasterNotRunningException(msg); + } + if (isDeadServer(sn)) { + throw new MasterNotRunningException(sn + " is dead."); + } + // Use the security info interface name as our stub key + String key = getStubKey(getServiceName(), sn.getHostname(), sn.getPort()); + connectionLock.putIfAbsent(key, key); + Object stub = null; + synchronized (connectionLock.get(key)) { + stub = stubs.get(key); + if (stub == null) { + BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout); + stub = makeStub(channel); + isMasterRunning(); + stubs.put(key, stub); + } + } + return stub; + } finally { + zkw.close(); + } + } + + /** + * Create a stub against the master. Retry if necessary. + * @return A stub to do intf against the master + * @throws org.apache.hadoop.hbase.MasterNotRunningException + */ + Object makeStub() throws IOException { + // The lock must be at the beginning to prevent multiple master creations + // (and leaks) in a multithread context + synchronized (masterAndZKLock) { + Exception exceptionCaught = null; + if (!closed) { + try { + return makeStubNoRetries(); + } catch (IOException e) { + exceptionCaught = e; + } catch (KeeperException e) { + exceptionCaught = e; + } catch (ServiceException e) { + exceptionCaught = e; + } + + throw new MasterNotRunningException(exceptionCaught); + } else { + throw new DoNotRetryIOException("Connection was closed while trying to get master"); + } + } + } + } + + /** + * Class to make a MasterServiceStubMaker stub. + */ + class MasterServiceStubMaker extends StubMaker { + private MasterProtos.MasterService.BlockingInterface stub; + @Override + protected String getServiceName() { + return MasterProtos.MasterService.getDescriptor().getName(); + } + + @Override + MasterProtos.MasterService.BlockingInterface makeStub() throws IOException { + return (MasterProtos.MasterService.BlockingInterface)super.makeStub(); + } + + @Override + protected Object makeStub(BlockingRpcChannel channel) { + this.stub = MasterProtos.MasterService.newBlockingStub(channel); + return this.stub; + } + + @Override + protected void isMasterRunning() throws ServiceException { + this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest()); + } + } + + @Override + public AdminProtos.AdminService.BlockingInterface getAdmin(final ServerName serverName) + throws IOException { + return getAdmin(serverName, false); + } + + @Override + // Nothing is done w/ the 'master' parameter. It is ignored. + public AdminProtos.AdminService.BlockingInterface getAdmin(final ServerName serverName, + final boolean master) + throws IOException { + if (isDeadServer(serverName)) { + throw new RegionServerStoppedException(serverName + " is dead."); + } + String key = getStubKey(AdminProtos.AdminService.BlockingInterface.class.getName(), + serverName.getHostname(), serverName.getPort()); + this.connectionLock.putIfAbsent(key, key); + AdminProtos.AdminService.BlockingInterface stub = null; + synchronized (this.connectionLock.get(key)) { + stub = (AdminProtos.AdminService.BlockingInterface)this.stubs.get(key); + if (stub == null) { + BlockingRpcChannel channel = + this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout); + stub = AdminProtos.AdminService.newBlockingStub(channel); + this.stubs.put(key, stub); + } + } + return stub; + } + + @Override + public ClientProtos.ClientService.BlockingInterface getClient(final ServerName sn) + throws IOException { + if (isDeadServer(sn)) { + throw new RegionServerStoppedException(sn + " is dead."); + } + String key = getStubKey(ClientProtos.ClientService.BlockingInterface.class.getName(), sn.getHostname(), + sn.getPort()); + this.connectionLock.putIfAbsent(key, key); + ClientProtos.ClientService.BlockingInterface stub = null; + synchronized (this.connectionLock.get(key)) { + stub = (ClientProtos.ClientService.BlockingInterface)this.stubs.get(key); + if (stub == null) { + BlockingRpcChannel channel = + this.rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout); + stub = ClientProtos.ClientService.newBlockingStub(channel); + // In old days, after getting stub/proxy, we'd make a call. We are not doing that here. + // Just fail on first actual call rather than in here on setup. + this.stubs.put(key, stub); + } + } + return stub; + } + + static String getStubKey(final String serviceName, final String rsHostname, int port) { + // Sometimes, servers go down and they come back up with the same hostname but a different + // IP address. Force a resolution of the rsHostname by trying to instantiate an + // InetSocketAddress, and this way we will rightfully get a new stubKey. + // Also, include the hostname in the key so as to take care of those cases where the + // DNS name is different but IP address remains the same. + InetAddress i = new InetSocketAddress(rsHostname, port).getAddress(); + String address = rsHostname; + if (i != null) { + address = i.getHostAddress() + "-" + rsHostname; + } + return serviceName + "@" + address + ":" + port; + } + + private ZooKeeperKeepAliveConnection keepAliveZookeeper; + private AtomicInteger keepAliveZookeeperUserCount = new AtomicInteger(0); + private boolean canCloseZKW = true; + + // keepAlive time, in ms. No reason to make it configurable. + private static final long keepAlive = 5 * 60 * 1000; + + /** + * Retrieve a shared ZooKeeperWatcher. You must close it it once you've have finished with it. + * @return The shared instance. Never returns null. + */ + ZooKeeperKeepAliveConnection getKeepAliveZooKeeperWatcher() + throws IOException { + synchronized (masterAndZKLock) { + if (keepAliveZookeeper == null) { + if (this.closed) { + throw new IOException(toString() + " closed"); + } + // We don't check that our link to ZooKeeper is still valid + // But there is a retry mechanism in the ZooKeeperWatcher itself + keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this); + } + keepAliveZookeeperUserCount.addAndGet(1); + keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE; + return keepAliveZookeeper; + } + } + + void releaseZooKeeperWatcher(final ZooKeeperWatcher zkw) { + if (zkw == null){ + return; + } + if (keepAliveZookeeperUserCount.addAndGet(-1) <= 0) { + keepZooKeeperWatcherAliveUntil = System.currentTimeMillis() + keepAlive; + } + } + + private void closeZooKeeperWatcher() { + synchronized (masterAndZKLock) { + if (keepAliveZookeeper != null) { + LOG.info("Closing zookeeper sessionid=0x" + + Long.toHexString( + keepAliveZookeeper.getRecoverableZooKeeper().getSessionId())); + keepAliveZookeeper.internalClose(); + keepAliveZookeeper = null; + } + keepAliveZookeeperUserCount.set(0); + } + } + + final MasterServiceState masterServiceState = new MasterServiceState(this); + + @Override + public MasterProtos.MasterService.BlockingInterface getMaster() throws MasterNotRunningException { + return getKeepAliveMasterService(); + } + + private void resetMasterServiceState(final MasterServiceState mss) { + mss.userCount++; + } + + @Override + public MasterKeepAliveConnection getKeepAliveMasterService() + throws MasterNotRunningException { + synchronized (masterAndZKLock) { + if (!isKeepAliveMasterConnectedAndRunning(this.masterServiceState)) { + MasterServiceStubMaker stubMaker = new MasterServiceStubMaker(); + try { + this.masterServiceState.stub = stubMaker.makeStub(); + } catch (MasterNotRunningException ex) { + throw ex; + } catch (IOException e) { + // rethrow as MasterNotRunningException so that we can keep the method sig + throw new MasterNotRunningException(e); + } + } + resetMasterServiceState(this.masterServiceState); + } + // Ugly delegation just so we can add in a Close method. + final MasterProtos.MasterService.BlockingInterface stub = this.masterServiceState.stub; + return new MasterKeepAliveConnection() { + MasterServiceState mss = masterServiceState; + @Override + public MasterProtos.AddColumnResponse addColumn(RpcController controller, MasterProtos.AddColumnRequest request) + throws ServiceException { + return stub.addColumn(controller, request); + } + + @Override + public MasterProtos.DeleteColumnResponse deleteColumn(RpcController controller, + MasterProtos.DeleteColumnRequest request) + throws ServiceException { + return stub.deleteColumn(controller, request); + } + + @Override + public MasterProtos.ModifyColumnResponse modifyColumn(RpcController controller, + MasterProtos.ModifyColumnRequest request) + throws ServiceException { + return stub.modifyColumn(controller, request); + } + + @Override + public MasterProtos.MoveRegionResponse moveRegion(RpcController controller, + MasterProtos.MoveRegionRequest request) throws ServiceException { + return stub.moveRegion(controller, request); + } + + @Override + public MasterProtos.DispatchMergingRegionsResponse dispatchMergingRegions( + RpcController controller, MasterProtos.DispatchMergingRegionsRequest request) + throws ServiceException { + return stub.dispatchMergingRegions(controller, request); + } + + @Override + public MasterProtos.AssignRegionResponse assignRegion(RpcController controller, + MasterProtos.AssignRegionRequest request) throws ServiceException { + return stub.assignRegion(controller, request); + } + + @Override + public MasterProtos.UnassignRegionResponse unassignRegion(RpcController controller, + MasterProtos.UnassignRegionRequest request) throws ServiceException { + return stub.unassignRegion(controller, request); + } + + @Override + public MasterProtos.OfflineRegionResponse offlineRegion(RpcController controller, + MasterProtos.OfflineRegionRequest request) throws ServiceException { + return stub.offlineRegion(controller, request); + } + + @Override + public MasterProtos.DeleteTableResponse deleteTable(RpcController controller, + MasterProtos.DeleteTableRequest request) throws ServiceException { + return stub.deleteTable(controller, request); + } + + @Override + public MasterProtos.TruncateTableResponse truncateTable(RpcController controller, + MasterProtos.TruncateTableRequest request) throws ServiceException { + return stub.truncateTable(controller, request); + } + + @Override + public MasterProtos.EnableTableResponse enableTable(RpcController controller, + MasterProtos.EnableTableRequest request) throws ServiceException { + return stub.enableTable(controller, request); + } + + @Override + public MasterProtos.DisableTableResponse disableTable(RpcController controller, + MasterProtos.DisableTableRequest request) throws ServiceException { + return stub.disableTable(controller, request); + } + + @Override + public MasterProtos.ModifyTableResponse modifyTable(RpcController controller, + MasterProtos.ModifyTableRequest request) throws ServiceException { + return stub.modifyTable(controller, request); + } + + @Override + public MasterProtos.CreateTableResponse createTable(RpcController controller, + MasterProtos.CreateTableRequest request) throws ServiceException { + return stub.createTable(controller, request); + } + + @Override + public MasterProtos.ShutdownResponse shutdown(RpcController controller, + MasterProtos.ShutdownRequest request) throws ServiceException { + return stub.shutdown(controller, request); + } + + @Override + public MasterProtos.StopMasterResponse stopMaster(RpcController controller, + MasterProtos.StopMasterRequest request) throws ServiceException { + return stub.stopMaster(controller, request); + } + + @Override + public MasterProtos.BalanceResponse balance(RpcController controller, + MasterProtos.BalanceRequest request) throws ServiceException { + return stub.balance(controller, request); + } + + @Override + public MasterProtos.SetBalancerRunningResponse setBalancerRunning( + RpcController controller, MasterProtos.SetBalancerRunningRequest request) + throws ServiceException { + return stub.setBalancerRunning(controller, request); + } + + @Override + public MasterProtos.RunCatalogScanResponse runCatalogScan(RpcController controller, + MasterProtos.RunCatalogScanRequest request) throws ServiceException { + return stub.runCatalogScan(controller, request); + } + + @Override + public MasterProtos.EnableCatalogJanitorResponse enableCatalogJanitor( + RpcController controller, MasterProtos.EnableCatalogJanitorRequest request) + throws ServiceException { + return stub.enableCatalogJanitor(controller, request); + } + + @Override + public MasterProtos.IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled( + RpcController controller, MasterProtos.IsCatalogJanitorEnabledRequest request) + throws ServiceException { + return stub.isCatalogJanitorEnabled(controller, request); + } + + @Override + public ClientProtos.CoprocessorServiceResponse execMasterService( + RpcController controller, ClientProtos.CoprocessorServiceRequest request) + throws ServiceException { + return stub.execMasterService(controller, request); + } + + @Override + public MasterProtos.SnapshotResponse snapshot(RpcController controller, + MasterProtos.SnapshotRequest request) throws ServiceException { + return stub.snapshot(controller, request); + } + + @Override + public MasterProtos.GetCompletedSnapshotsResponse getCompletedSnapshots( + RpcController controller, MasterProtos.GetCompletedSnapshotsRequest request) + throws ServiceException { + return stub.getCompletedSnapshots(controller, request); + } + + @Override + public MasterProtos.DeleteSnapshotResponse deleteSnapshot(RpcController controller, + MasterProtos.DeleteSnapshotRequest request) throws ServiceException { + return stub.deleteSnapshot(controller, request); + } + + @Override + public MasterProtos.IsSnapshotDoneResponse isSnapshotDone(RpcController controller, + MasterProtos.IsSnapshotDoneRequest request) throws ServiceException { + return stub.isSnapshotDone(controller, request); + } + + @Override + public MasterProtos.RestoreSnapshotResponse restoreSnapshot( + RpcController controller, MasterProtos.RestoreSnapshotRequest request) + throws ServiceException { + return stub.restoreSnapshot(controller, request); + } + + @Override + public MasterProtos.IsRestoreSnapshotDoneResponse isRestoreSnapshotDone( + RpcController controller, MasterProtos.IsRestoreSnapshotDoneRequest request) + throws ServiceException { + return stub.isRestoreSnapshotDone(controller, request); + } + + @Override + public MasterProtos.ExecProcedureResponse execProcedure( + RpcController controller, MasterProtos.ExecProcedureRequest request) + throws ServiceException { + return stub.execProcedure(controller, request); + } + + @Override + public MasterProtos.ExecProcedureResponse execProcedureWithRet( + RpcController controller, MasterProtos.ExecProcedureRequest request) + throws ServiceException { + return stub.execProcedureWithRet(controller, request); + } + + @Override + public MasterProtos.IsProcedureDoneResponse isProcedureDone(RpcController controller, + MasterProtos.IsProcedureDoneRequest request) throws ServiceException { + return stub.isProcedureDone(controller, request); + } + + @Override + public MasterProtos.IsMasterRunningResponse isMasterRunning( + RpcController controller, MasterProtos.IsMasterRunningRequest request) + throws ServiceException { + return stub.isMasterRunning(controller, request); + } + + @Override + public MasterProtos.ModifyNamespaceResponse modifyNamespace(RpcController controller, + MasterProtos.ModifyNamespaceRequest request) + throws ServiceException { + return stub.modifyNamespace(controller, request); + } + + @Override + public MasterProtos.CreateNamespaceResponse createNamespace( + RpcController controller, MasterProtos.CreateNamespaceRequest request) throws ServiceException { + return stub.createNamespace(controller, request); + } + + @Override + public MasterProtos.DeleteNamespaceResponse deleteNamespace( + RpcController controller, MasterProtos.DeleteNamespaceRequest request) throws ServiceException { + return stub.deleteNamespace(controller, request); + } + + @Override + public MasterProtos.GetNamespaceDescriptorResponse getNamespaceDescriptor(RpcController controller, + MasterProtos.GetNamespaceDescriptorRequest request) throws ServiceException { + return stub.getNamespaceDescriptor(controller, request); + } + + @Override + public MasterProtos.ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller, + MasterProtos.ListNamespaceDescriptorsRequest request) throws ServiceException { + return stub.listNamespaceDescriptors(controller, request); + } + + @Override + public MasterProtos.ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace( + RpcController controller, MasterProtos.ListTableDescriptorsByNamespaceRequest request) + throws ServiceException { + return stub.listTableDescriptorsByNamespace(controller, request); + } + + @Override + public MasterProtos.ListTableNamesByNamespaceResponse listTableNamesByNamespace( + RpcController controller, MasterProtos.ListTableNamesByNamespaceRequest request) + throws ServiceException { + return stub.listTableNamesByNamespace(controller, request); + } + + @Override + public MasterProtos.GetTableStateResponse getTableState( + RpcController controller, MasterProtos.GetTableStateRequest request) + throws ServiceException { + return stub.getTableState(controller, request); + } + + @Override + public void close() { + release(this.mss); + } + + @Override + public MasterProtos.GetSchemaAlterStatusResponse getSchemaAlterStatus( + RpcController controller, MasterProtos.GetSchemaAlterStatusRequest request) + throws ServiceException { + return stub.getSchemaAlterStatus(controller, request); + } + + @Override + public MasterProtos.GetTableDescriptorsResponse getTableDescriptors( + RpcController controller, MasterProtos.GetTableDescriptorsRequest request) + throws ServiceException { + return stub.getTableDescriptors(controller, request); + } + + @Override + public MasterProtos.GetTableNamesResponse getTableNames( + RpcController controller, MasterProtos.GetTableNamesRequest request) + throws ServiceException { + return stub.getTableNames(controller, request); + } + + @Override + public MasterProtos.GetClusterStatusResponse getClusterStatus( + RpcController controller, MasterProtos.GetClusterStatusRequest request) + throws ServiceException { + return stub.getClusterStatus(controller, request); + } + + @Override + public MasterProtos.SetQuotaResponse setQuota( + RpcController controller, MasterProtos.SetQuotaRequest request) + throws ServiceException { + return stub.setQuota(controller, request); + } + + @Override + public MasterProtos.MajorCompactionTimestampResponse getLastMajorCompactionTimestamp( + RpcController controller, MasterProtos.MajorCompactionTimestampRequest request) + throws ServiceException { + return stub.getLastMajorCompactionTimestamp(controller, request); + } + + @Override + public MasterProtos.MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion( + RpcController controller, MasterProtos.MajorCompactionTimestampForRegionRequest request) + throws ServiceException { + return stub.getLastMajorCompactionTimestampForRegion(controller, request); + } + }; + } + + + private static void release(MasterServiceState mss) { + if (mss != null && mss.connection != null) { + ((HConnectionImplementation)mss.connection).releaseMaster(mss); + } + } + + private boolean isKeepAliveMasterConnectedAndRunning(MasterServiceState mss) { + if (mss.getStub() == null){ + return false; + } + try { + return mss.isMasterRunning(); + } catch (UndeclaredThrowableException e) { + // It's somehow messy, but we can receive exceptions such as + // java.net.ConnectException but they're not declared. So we catch it... + LOG.info("Master connection is not running anymore", e.getUndeclaredThrowable()); + return false; + } catch (ServiceException se) { + LOG.warn("Checking master connection", se); + return false; + } + } + + void releaseMaster(MasterServiceState mss) { + if (mss.getStub() == null) return; + synchronized (masterAndZKLock) { + --mss.userCount; + } + } + + private void closeMasterService(MasterServiceState mss) { + if (mss.getStub() != null) { + LOG.info("Closing master protocol: " + mss); + mss.clearStub(); + } + mss.userCount = 0; + } + + /** + * Immediate close of the shared master. Can be by the delayed close or when closing the + * connection itself. + */ + private void closeMaster() { + synchronized (masterAndZKLock) { + closeMasterService(masterServiceState); + } + } + + void updateCachedLocation(HRegionInfo hri, ServerName source, + ServerName serverName, long seqNum) { + HRegionLocation newHrl = new HRegionLocation(hri, serverName, seqNum); + cacheLocation(hri.getTable(), source, newHrl); + } + + @Override + public void deleteCachedRegionLocation(final HRegionLocation location) { + metaCache.clearCache(location); + } + + @Override + public void updateCachedLocations(final TableName tableName, byte[] rowkey, + final Object exception, final HRegionLocation source) { + assert source != null; + updateCachedLocations(tableName, source.getRegionInfo().getRegionName() + , rowkey, exception, source.getServerName()); + } + + /** + * Update the location with the new value (if the exception is a RegionMovedException) + * or delete it from the cache. Does nothing if we can be sure from the exception that + * the location is still accurate, or if the cache has already been updated. + * @param exception an object (to simplify user code) on which we will try to find a nested + * or wrapped or both RegionMovedException + * @param source server that is the source of the location update. + */ + @Override + public void updateCachedLocations(final TableName tableName, byte[] regionName, byte[] rowkey, + final Object exception, final ServerName source) { + if (rowkey == null || tableName == null) { + LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) + + ", tableName=" + (tableName == null ? "null" : tableName)); + return; + } + + if (source == null) { + // This should not happen, but let's secure ourselves. + return; + } + + if (regionName == null) { + // we do not know which region, so just remove the cache entry for the row and server + metaCache.clearCache(tableName, rowkey, source); + return; + } + + // Is it something we have already updated? + final RegionLocations oldLocations = getCachedLocation(tableName, rowkey); + HRegionLocation oldLocation = null; + if (oldLocations != null) { + oldLocation = oldLocations.getRegionLocationByRegionName(regionName); + } + if (oldLocation == null || !source.equals(oldLocation.getServerName())) { + // There is no such location in the cache (it's been removed already) or + // the cache has already been refreshed with a different location. => nothing to do + return; + } + + HRegionInfo regionInfo = oldLocation.getRegionInfo(); + Throwable cause = ConnectionManager.findException(exception); + if (cause != null) { + if (cause instanceof RegionTooBusyException || cause instanceof RegionOpeningException) { + // We know that the region is still on this region server + return; + } + + if (cause instanceof RegionMovedException) { + RegionMovedException rme = (RegionMovedException) cause; + if (LOG.isTraceEnabled()) { + LOG.trace("Region " + regionInfo.getRegionNameAsString() + " moved to " + + rme.getHostname() + ":" + rme.getPort() + + " according to " + source.getHostAndPort()); + } + // We know that the region is not anymore on this region server, but we know + // the new location. + updateCachedLocation( + regionInfo, source, rme.getServerName(), rme.getLocationSeqNum()); + return; + } + } + + // If we're here, it means that can cannot be sure about the location, so we remove it from + // the cache. Do not send the source because source can be a new server in the same host:port + metaCache.clearCache(regionInfo); + } + + @Override + public void updateCachedLocations(final byte[] tableName, byte[] rowkey, + final Object exception, final HRegionLocation source) { + updateCachedLocations(TableName.valueOf(tableName), rowkey, exception, source); + } + + /** + * @deprecated since 0.96 - Use {@link org.apache.hadoop.hbase.client.HTableInterface#batch} instead + */ + @Override + @Deprecated + public void processBatch(List list, + final TableName tableName, + ExecutorService pool, + Object[] results) throws IOException, InterruptedException { + // This belongs in HTable!!! Not in here. St.Ack + + // results must be the same size as list + if (results.length != list.size()) { + throw new IllegalArgumentException( + "argument results must be the same size as argument list"); + } + processBatchCallback(list, tableName, pool, results, null); + } + + /** + * @deprecated Unsupported API + */ + @Override + @Deprecated + public void processBatch(List list, + final byte[] tableName, + ExecutorService pool, + Object[] results) throws IOException, InterruptedException { + processBatch(list, TableName.valueOf(tableName), pool, results); + } + + /** + * Send the queries in parallel on the different region servers. Retries on failures. + * If the method returns it means that there is no error, and the 'results' array will + * contain no exception. On error, an exception is thrown, and the 'results' array will + * contain results and exceptions. + * @deprecated since 0.96 - Use {@link org.apache.hadoop.hbase.client.HTable#processBatchCallback} instead + */ + @Override + @Deprecated + public void processBatchCallback( + List list, + TableName tableName, + ExecutorService pool, + Object[] results, + Batch.Callback callback) + throws IOException, InterruptedException { + + AsyncProcess.AsyncRequestFuture ars = this.asyncProcess.submitAll( + pool, tableName, list, callback, results); + ars.waitUntilDone(); + if (ars.hasError()) { + throw ars.getErrors(); + } + } + + /** + * @deprecated Unsupported API + */ + @Override + @Deprecated + public void processBatchCallback( + List list, + byte[] tableName, + ExecutorService pool, + Object[] results, + Batch.Callback callback) + throws IOException, InterruptedException { + processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback); + } + + // For tests to override. + protected AsyncProcess createAsyncProcess(Configuration conf) { + // No default pool available. + return new AsyncProcess(this, conf, this.batchPool, + RpcRetryingCallerFactory.instantiate(conf, this.getStatisticsTracker()), false, + RpcControllerFactory.instantiate(conf)); + } + + @Override + public AsyncProcess getAsyncProcess() { + return asyncProcess; + } + + @Override + public ServerStatisticTracker getStatisticsTracker() { + return this.stats; + } + + @Override + public ClientBackoffPolicy getBackoffPolicy() { + return this.backoffPolicy; + } + + /* + * Return the number of cached region for a table. It will only be called + * from a unit test. + */ + @VisibleForTesting + int getNumberOfCachedRegionLocations(final TableName tableName) { + return metaCache.getNumberOfCachedRegionLocations(tableName); + } + + /** + * @deprecated always return false since 0.99 + */ + @Override + @Deprecated + public void setRegionCachePrefetch(final TableName tableName, final boolean enable) { + } + + /** + * @deprecated always return false since 0.99 + */ + @Override + @Deprecated + public void setRegionCachePrefetch(final byte[] tableName, + final boolean enable) { + } + + /** + * @deprecated always return false since 0.99 + */ + @Override + @Deprecated + public boolean getRegionCachePrefetch(TableName tableName) { + return false; + } + + /** + * @deprecated always return false since 0.99 + */ + @Override + @Deprecated + public boolean getRegionCachePrefetch(byte[] tableName) { + return false; + } + + @Override + public void abort(final String msg, Throwable t) { + if (t instanceof KeeperException.SessionExpiredException + && keepAliveZookeeper != null) { + synchronized (masterAndZKLock) { + if (keepAliveZookeeper != null) { + LOG.warn("This client just lost it's session with ZooKeeper," + + " closing it." + + " It will be recreated next time someone needs it", t); + closeZooKeeperWatcher(); + } + } + } else { + if (t != null) { + LOG.fatal(msg, t); + } else { + LOG.fatal(msg); + } + this.aborted = true; + close(); + this.closed = true; + } + } + + @Override + public boolean isClosed() { + return this.closed; + } + + @Override + public boolean isAborted(){ + return this.aborted; + } + + @Override + public int getCurrentNrHRS() throws IOException { + return this.registry.getCurrentNrHRS(); + } + + /** + * Increment this client's reference count. + */ + void incCount() { + ++refCount; + } + + /** + * Decrement this client's reference count. + */ + void decCount() { + if (refCount > 0) { + --refCount; + } + } + + /** + * Return if this client has no reference + * + * @return true if this client has no reference; false otherwise + */ + boolean isZeroReference() { + return refCount == 0; + } + + void internalClose() { + if (this.closed) { + return; + } + closeMaster(); + shutdownPools(); + this.closed = true; + closeZooKeeperWatcher(); + this.stubs.clear(); + if (clusterStatusListener != null) { + clusterStatusListener.close(); + } + if (rpcClient != null) { + rpcClient.close(); + } + } + + @Override + public void close() { + if (managed) { + if (aborted) { + ConnectionManager.deleteStaleConnection(this); + } else { + ConnectionManager.deleteConnection(this, false); + } + } else { + internalClose(); + } + } + + /** + * Close the connection for good, regardless of what the current value of + * {@link #refCount} is. Ideally, {@link #refCount} should be zero at this + * point, which would be the case if all of its consumers close the + * connection. However, on the off chance that someone is unable to close + * the connection, perhaps because it bailed out prematurely, the method + * below will ensure that this {@link org.apache.hadoop.hbase.client.HConnection} instance is cleaned up. + * Caveat: The JVM may take an unknown amount of time to call finalize on an + * unreachable object, so our hope is that every consumer cleans up after + * itself, like any good citizen. + */ + @Override + protected void finalize() throws Throwable { + super.finalize(); + // Pretend as if we are about to release the last remaining reference + refCount = 1; + close(); + } + + /** + * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#listTables()} instead + */ + @Deprecated + @Override + public HTableDescriptor[] listTables() throws IOException { + MasterKeepAliveConnection master = getKeepAliveMasterService(); + try { + MasterProtos.GetTableDescriptorsRequest req = + RequestConverter.buildGetTableDescriptorsRequest((List)null); + return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req)); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } finally { + master.close(); + } + } + + /** + * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#listTableNames()} instead + */ + @Deprecated + @Override + public String[] getTableNames() throws IOException { + TableName[] tableNames = listTableNames(); + String[] result = new String[tableNames.length]; + for (int i = 0; i < tableNames.length; i++) { + result[i] = tableNames[i].getNameAsString(); + } + return result; + } + + /** + * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#listTableNames()} instead + */ + @Deprecated + @Override + public TableName[] listTableNames() throws IOException { + MasterKeepAliveConnection master = getKeepAliveMasterService(); + try { + return ProtobufUtil.getTableNameArray(master.getTableNames(null, + MasterProtos.GetTableNamesRequest.newBuilder().build()) + .getTableNamesList()); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } finally { + master.close(); + } + } + + /** + * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)} instead + */ + @Deprecated + @Override + public HTableDescriptor[] getHTableDescriptorsByTableName( + List tableNames) throws IOException { + if (tableNames == null || tableNames.isEmpty()) return new HTableDescriptor[0]; + MasterKeepAliveConnection master = getKeepAliveMasterService(); + try { + MasterProtos.GetTableDescriptorsRequest req = + RequestConverter.buildGetTableDescriptorsRequest(tableNames); + return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req)); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } finally { + master.close(); + } + } + + /** + * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)} instead + */ + @Deprecated + @Override + public HTableDescriptor[] getHTableDescriptors( + List names) throws IOException { + List tableNames = new ArrayList(names.size()); + for(String name : names) { + tableNames.add(TableName.valueOf(name)); + } + + return getHTableDescriptorsByTableName(tableNames); + } + + @Override + public NonceGenerator getNonceGenerator() { + return nonceGenerator; + } + + /** + * Connects to the master to get the table descriptor. + * @param tableName table name + * @throws java.io.IOException if the connection to master fails or if the table + * is not found. + * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)} instead + */ + @Deprecated + @Override + public HTableDescriptor getHTableDescriptor(final TableName tableName) + throws IOException { + if (tableName == null) return null; + MasterKeepAliveConnection master = getKeepAliveMasterService(); + MasterProtos.GetTableDescriptorsResponse htds; + try { + MasterProtos.GetTableDescriptorsRequest req = + RequestConverter.buildGetTableDescriptorsRequest(tableName); + htds = master.getTableDescriptors(null, req); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } finally { + master.close(); + } + if (!htds.getTableSchemaList().isEmpty()) { + return HTableDescriptor.convert(htds.getTableSchemaList().get(0)); + } + throw new TableNotFoundException(tableName.getNameAsString()); + } + + /** + * @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)} instead + */ + @Deprecated + @Override + public HTableDescriptor getHTableDescriptor(final byte[] tableName) + throws IOException { + return getHTableDescriptor(TableName.valueOf(tableName)); + } + + @Override + public TableState getTableState(TableName tableName) throws IOException { + ClusterConnection conn = ConnectionManager.getConnectionInternal(getConfiguration()); + TableState tableState = MetaTableAccessor.getTableState(conn, tableName); + if (tableState == null) + throw new TableNotFoundException(tableName); + return tableState; + } + + @Override + public RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf) { + return RpcRetryingCallerFactory + .instantiate(conf, this.interceptor, this.getStatisticsTracker()); + } + + @Override + public boolean isManaged() { + return managed; + } +} 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 2a8063a..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 @@ -319,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/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/client/ZooKeeperKeepAliveConnection.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java index 04fd20f..7c4ce67 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java @@ -39,14 +39,14 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; class ZooKeeperKeepAliveConnection extends ZooKeeperWatcher{ ZooKeeperKeepAliveConnection( Configuration conf, String descriptor, - ConnectionManager.HConnectionImplementation conn) throws IOException { + HConnectionImplementation conn) throws IOException { super(conf, descriptor, conn); } @Override public void close() { if (this.abortable != null) { - ((ConnectionManager.HConnectionImplementation)abortable).releaseZooKeeperWatcher(this); + ((HConnectionImplementation)abortable).releaseZooKeeperWatcher(this); } } diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java index 26aca18..3b8ef48 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java @@ -37,14 +37,14 @@ import org.apache.zookeeper.KeeperException; class ZooKeeperRegistry implements Registry { static final Log LOG = LogFactory.getLog(ZooKeeperRegistry.class); // Needs an instance of hci to function. Set after construct this instance. - ConnectionManager.HConnectionImplementation hci; + HConnectionImplementation hci; @Override public void init(Connection connection) { - if (!(connection instanceof ConnectionManager.HConnectionImplementation)) { + if (!(connection instanceof HConnectionImplementation)) { throw new RuntimeException("This registry depends on HConnectionImplementation"); } - this.hci = (ConnectionManager.HConnectionImplementation)connection; + this.hci = (HConnectionImplementation)connection; } @Override 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/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java index aa41939..efccb6c 100644 --- hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java +++ hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java @@ -338,7 +338,7 @@ public class TestAsyncProcess { /** * Returns our async process. */ - static class MyConnectionImpl extends ConnectionManager.HConnectionImplementation { + static class MyConnectionImpl extends HConnectionImplementation { final AtomicInteger nbThreads = new AtomicInteger(0); diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index 23eed1d..9480199 100644 --- hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -262,7 +262,7 @@ public class TestClientNoCluster extends Configured implements Tool { * Override to shutdown going to zookeeper for cluster id and meta location. */ static class ScanOpenNextThenExceptionThenRecoverConnection - extends ConnectionManager.HConnectionImplementation { + extends HConnectionImplementation { final ClientService.BlockingInterface stub; ScanOpenNextThenExceptionThenRecoverConnection(Configuration conf, @@ -294,7 +294,7 @@ public class TestClientNoCluster extends Configured implements Tool { * Override to shutdown going to zookeeper for cluster id and meta location. */ static class RegionServerStoppedOnScannerOpenConnection - extends ConnectionManager.HConnectionImplementation { + extends HConnectionImplementation { final ClientService.BlockingInterface stub; RegionServerStoppedOnScannerOpenConnection(Configuration conf, boolean managed, @@ -326,7 +326,7 @@ public class TestClientNoCluster extends Configured implements Tool { * Override to check we are setting rpc timeout right. */ static class RpcTimeoutConnection - extends ConnectionManager.HConnectionImplementation { + extends HConnectionImplementation { final ClientService.BlockingInterface stub; RpcTimeoutConnection(Configuration conf, boolean managed, ExecutorService pool, User user) @@ -353,7 +353,7 @@ public class TestClientNoCluster extends Configured implements Tool { * Fake many regionservers and many regions on a connection implementation. */ static class ManyServersManyRegionsConnection - extends ConnectionManager.HConnectionImplementation { + extends HConnectionImplementation { // All access should be synchronized final Map serversByClient; diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java index 78d718e..0ee84ff 100644 --- hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java +++ hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java @@ -72,8 +72,8 @@ public class TestSnapshotFromAdmin { + "- further testing won't prove anything.", time < ignoreExpectedTime); // setup the mocks - ConnectionManager.HConnectionImplementation mockConnection = Mockito - .mock(ConnectionManager.HConnectionImplementation.class); + HConnectionImplementation mockConnection = Mockito + .mock(HConnectionImplementation.class); Configuration conf = HBaseConfiguration.create(); // setup the conf to match the expected properties conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries); @@ -119,8 +119,8 @@ public class TestSnapshotFromAdmin { */ @Test public void testValidateSnapshotName() throws Exception { - ConnectionManager.HConnectionImplementation mockConnection = Mockito - .mock(ConnectionManager.HConnectionImplementation.class); + HConnectionImplementation mockConnection = Mockito + .mock(HConnectionImplementation.class); Configuration conf = HBaseConfiguration.create(); Mockito.when(mockConnection.getConfiguration()).thenReturn(conf); Admin admin = new HBaseAdmin(mockConnection); 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 9d07479..b82c750 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 @@ -44,7 +44,6 @@ 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; @@ -450,7 +449,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; 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/CoprocessorHConnection.java hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java index 1db90fb..6ccc227 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java @@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RegionServerServices; 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..28f9f39 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,17 @@ 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 8fa711c..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,10 +27,11 @@ 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.Table; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; @@ -148,7 +149,7 @@ public class ConnectionCache { */ 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..96e0d48 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,16 @@ 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 { + if (hConnection != null) { + 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/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 0d05c68..0415641 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 @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.mockito.Mockito; @@ -39,15 +38,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 - * will stick around; this is probably not what you want. + * {@link Connection#close()} else it will stick around; this is probably not what you want. * @param conf configuration * @return HConnection object for conf * @throws ZooKeeperConnectionException @@ -71,9 +69,8 @@ public class HConnectionTestingUtility { * Calls {@link #getMockedConnection(Configuration)} and then mocks a few * 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 - * will stick around; this is probably not what you want. + * connection when done by calling {@link Connection#close()} else it will stick around; + * this is probably not what you want. * * @param conf Configuration to use * @param admin An AdminProtocol; can be null but is usually @@ -92,8 +89,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)} - * when done with this mocked Connection. + * {@link Connection#close()} when done with this mocked Connection. * @throws IOException */ public static ClusterConnection getMockedConnectionAndDecorate(final Configuration conf, @@ -146,8 +142,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 - * will stick around; this is probably not what you want. + * {@link Connection#close()} else it will stick around; this is probably not what you want. * @param conf configuration * @return HConnection object for conf * @throws ZooKeeperConnectionException 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..a4ceaa8 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 @@ -285,96 +285,6 @@ public class TestFromClientSide { table.close(); } - /** - * @deprecated Tests deprecated functionality. Remove when we are past 1.0. - * @throws Exception - */ - @Deprecated - @Test - public void testSharedZooKeeper() throws Exception { - Configuration newConfig = new Configuration(TEST_UTIL.getConfiguration()); - newConfig.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "12345"); - - // First with a simple ZKW - ZooKeeperWatcher z0 = new ZooKeeperWatcher( - newConfig, "hconnection", new Abortable() { - @Override public void abort(String why, Throwable e) {} - @Override public boolean isAborted() {return false;} - }); - z0.getRecoverableZooKeeper().getZooKeeper().exists("/oldZooKeeperWatcher", false); - z0.close(); - - // Then a ZooKeeperKeepAliveConnection - ConnectionManager.HConnectionImplementation connection1 = - (ConnectionManager.HConnectionImplementation) - HConnectionManager.getConnection(newConfig); - - ZooKeeperKeepAliveConnection z1 = connection1.getKeepAliveZooKeeperWatcher(); - z1.getRecoverableZooKeeper().getZooKeeper().exists("/z1", false); - - z1.close(); - - // will still work, because the real connection is not closed yet - // Not do be done in real code - z1.getRecoverableZooKeeper().getZooKeeper().exists("/z1afterclose", false); - - - ZooKeeperKeepAliveConnection z2 = connection1.getKeepAliveZooKeeperWatcher(); - assertTrue( - "ZooKeeperKeepAliveConnection equals on same connection", z1 == z2); - - - - Configuration newConfig2 = new Configuration(TEST_UTIL.getConfiguration()); - newConfig2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "6789"); - ConnectionManager.HConnectionImplementation connection2 = - (ConnectionManager.HConnectionImplementation) - HConnectionManager.getConnection(newConfig2); - - assertTrue("connections should be different ", connection1 != connection2); - - ZooKeeperKeepAliveConnection z3 = connection2.getKeepAliveZooKeeperWatcher(); - assertTrue( - "ZooKeeperKeepAliveConnection should be different" + - " on different connections", z1 != z3); - - // Bypass the private access - Method m = ConnectionManager.HConnectionImplementation.class. - getDeclaredMethod("closeZooKeeperWatcher"); - m.setAccessible(true); - m.invoke(connection2); - - ZooKeeperKeepAliveConnection z4 = connection2.getKeepAliveZooKeeperWatcher(); - assertTrue( - "ZooKeeperKeepAliveConnection should be recreated" + - " when previous connections was closed" - , z3 != z4); - - - z2.getRecoverableZooKeeper().getZooKeeper().exists("/z2", false); - z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false); - - - HConnectionManager.deleteConnection(newConfig); - try { - z2.getRecoverableZooKeeper().getZooKeeper().exists("/z2", false); - assertTrue("We should not have a valid connection for z2", false); - } catch (Exception e){ - } - - z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false); - // We expect success here. - - - HConnectionManager.deleteConnection(newConfig2); - try { - z4.getRecoverableZooKeeper().getZooKeeper().exists("/z4", false); - assertTrue("We should not have a valid connection for z4", false); - } catch (Exception e){ - } - } - - /** * Verifies that getConfiguration returns the same Configuration object used * to create the HTable instance. @@ -4127,7 +4037,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 5d284a2..1fba87d 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 @@ -29,7 +29,6 @@ import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.net.SocketTimeoutException; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -56,7 +55,6 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; @@ -155,8 +153,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()); @@ -388,8 +386,8 @@ public class TestHCM { }); ServerName sn = table.getRegionLocation(ROW).getServerName(); - ConnectionManager.HConnectionImplementation conn = - (ConnectionManager.HConnectionImplementation) table.getConnection(); + HConnectionImplementation conn = + (HConnectionImplementation) table.getConnection(); RpcClient rpcClient = conn.getRpcClient(); LOG.info("Going to cancel connections. connection=" + conn.toString() + ", sn=" + sn); @@ -537,7 +535,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 +578,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(); @@ -609,8 +607,8 @@ public class TestHCM { Put put = new Put(ROW); put.add(FAM_NAM, ROW, ROW); table.put(put); - ConnectionManager.HConnectionImplementation conn = - (ConnectionManager.HConnectionImplementation)table.getConnection(); + HConnectionImplementation conn = + (HConnectionImplementation)table.getConnection(); assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW)); @@ -810,8 +808,8 @@ public class TestHCM { Put put = new Put(ROW); put.add(FAM_NAM, ROW, ROW); table.put(put); - ConnectionManager.HConnectionImplementation conn = - (ConnectionManager.HConnectionImplementation)table.getConnection(); + HConnectionImplementation conn = + (HConnectionImplementation)table.getConnection(); HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation(); assertNotNull(location); @@ -861,7 +859,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( @@ -877,59 +875,6 @@ public class TestHCM { } } - /** - * Makes sure that there is no leaking of - * {@link ConnectionManager.HConnectionImplementation} in the {@link HConnectionManager} - * class. - * @deprecated Tests deprecated functionality. Remove in 1.0. - */ - @Deprecated - @Test - public void testConnectionUniqueness() throws Exception { - int zkmaxconnections = TEST_UTIL.getConfiguration(). - getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, - HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS); - // Test up to a max that is < the maximum number of zk connections. If we - // go above zk connections, we just fall into cycle where we are failing - // to set up a session and test runs for a long time. - int maxConnections = Math.min(zkmaxconnections - 1, 20); - List connections = new ArrayList(maxConnections); - Connection previousConnection = null; - try { - for (int i = 0; i < maxConnections; i++) { - // set random key to differentiate the connection from previous ones - Configuration configuration = new Configuration(TEST_UTIL.getConfiguration()); - configuration.set("some_key", String.valueOf(_randy.nextInt())); - configuration.set(HConstants.HBASE_CLIENT_INSTANCE_ID, - String.valueOf(_randy.nextInt())); - LOG.info("The hash code of the current configuration is: " - + configuration.hashCode()); - HConnection currentConnection = - HConnectionManager.getConnection(configuration); - if (previousConnection != null) { - assertTrue("Got the same connection even though its key changed!", - previousConnection != currentConnection); - } - // change the configuration, so that it is no longer reachable from the - // client's perspective. However, since its part of the LRU doubly linked - // list, it will eventually get thrown out, at which time it should also - // close the corresponding {@link HConnection}. - configuration.set("other_key", String.valueOf(_randy.nextInt())); - - previousConnection = currentConnection; - LOG.info("The current HConnectionManager#HBASE_INSTANCES cache size is: " - + getHConnectionManagerCacheSize()); - Thread.sleep(50); - connections.add(currentConnection); - } - } finally { - for (Connection c: connections) { - // Clean up connections made so we don't interfere w/ subsequent tests. - HConnectionManager.deleteConnection(c.getConfiguration()); - } - } - } - @Test public void testClosing() throws Exception { Configuration configuration = @@ -937,36 +882,26 @@ public class TestHCM { configuration.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(_randy.nextInt())); + // as connection caching is going away, now we're just testing + // that closed connection does actually get closed. + Connection c1 = ConnectionFactory.createConnection(configuration); - // We create two connections with the same key. Connection c2 = ConnectionFactory.createConnection(configuration); + // no caching, different connections + assertTrue(c1 != c2); - Connection c3 = HConnectionManager.getConnection(configuration); - Connection c4 = HConnectionManager.getConnection(configuration); - assertTrue(c3 == c4); - + // closing independently c1.close(); assertTrue(c1.isClosed()); assertFalse(c2.isClosed()); - assertFalse(c3.isClosed()); - c3.close(); - // still a reference left - assertTrue(c3.isClosed()); - - Connection c5 = HConnectionManager.getConnection(configuration); - assertTrue(c5 != c3); - - assertFalse(c2.isClosed()); c2.close(); assertTrue(c2.isClosed()); - c5.close(); - assertTrue(c5.isClosed()); } /** * Trivial test to verify that nobody messes with - * {@link HConnectionManager#createConnection(Configuration)} + * {@link ConnectionFactory#createConnection(Configuration)} */ @Test public void testCreateConnection() throws Exception { @@ -977,7 +912,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); } @@ -998,7 +933,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(); } @@ -1019,8 +954,8 @@ public class TestHCM { public void testMulti() throws Exception { HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME3, FAM_NAM); try { - ConnectionManager.HConnectionImplementation conn = - ( ConnectionManager.HConnectionImplementation)table.getConnection(); + HConnectionImplementation conn = + (HConnectionImplementation)table.getConnection(); // We're now going to move the region and check that it works for the client // First a new put to add the location in the cache @@ -1222,9 +1157,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/client/TestMetaWithReplicas.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java index b83dc81..8684dea 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java @@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java index abea699..4237e5d 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java @@ -567,7 +567,7 @@ public class TestMultiParallel { validateResult(result, QUALIFIER, Bytes.toBytes((numRequests / 2) + 1L)); table.close(); } finally { - ConnectionManager.injectNonceGeneratorForTesting((ClusterConnection)connection, oldCnm); + HConnectionImplementation.injectNonceGeneratorForTesting((ClusterConnection) connection, oldCnm); } } 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/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/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