You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by se...@apache.org on 2014/02/10 05:16:35 UTC

svn commit: r1566501 [5/5] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/test/java/org/apache/hadoop/hbase/client/ hbase-server/src/main/java/org/apache/hadoop/hbase/client/ hbase-server/src/main/java/or...

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java Mon Feb 10 04:16:34 2014
@@ -69,6 +69,7 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
 
@@ -117,7 +118,7 @@ import com.google.protobuf.ServiceExcept
 @InterfaceStability.Stable
 public class HTable implements HTableInterface {
   private static final Log LOG = LogFactory.getLog(HTable.class);
-  protected HConnection connection;
+  protected ClusterConnection connection;
   private final TableName tableName;
   private volatile Configuration configuration;
   protected List<Row> writeAsyncBuffer = new LinkedList<Row>();
@@ -189,7 +190,7 @@ public class HTable implements HTableInt
       this.connection = null;
       return;
     }
-    this.connection = HConnectionManager.getConnection(conf);
+    this.connection = ConnectionManager.getConnectionInternal(conf);
     this.configuration = conf;
 
     this.pool = getDefaultExecutor(conf);
@@ -203,12 +204,14 @@ public class HTable implements HTableInt
    * @param tableName Name of the table.
    * @param connection HConnection to be used.
    * @throws IOException if a remote or network exception occurs
+   * @deprecated Do not use.
    */
+  @Deprecated
   public HTable(TableName tableName, HConnection connection) throws IOException {
     this.tableName = tableName;
     this.cleanupPoolOnClose = true;
     this.cleanupConnectionOnClose = false;
-    this.connection = connection;
+    this.connection = (ClusterConnection)connection;
     this.configuration = connection.getConfiguration();
 
     this.pool = getDefaultExecutor(this.configuration);
@@ -263,7 +266,7 @@ public class HTable implements HTableInt
    */
   public HTable(Configuration conf, final TableName tableName, final ExecutorService pool)
       throws IOException {
-    this.connection = HConnectionManager.getConnection(conf);
+    this.connection = ConnectionManager.getConnectionInternal(conf);
     this.configuration = conf;
     this.pool = pool;
     this.tableName = tableName;
@@ -282,25 +285,35 @@ public class HTable implements HTableInt
    * @param tableName Name of the table.
    * @param connection HConnection to be used.
    * @param pool ExecutorService to be used.
-   * @throws IOException if a remote or network exception occurs
+   * @throws IOException if a remote or network exception occurs.
+   * @deprecated Do not use, internal ctor.
    */
+  @Deprecated
   public HTable(final byte[] tableName, final HConnection connection,
       final ExecutorService pool) throws IOException {
     this(TableName.valueOf(tableName), connection, pool);
   }
 
+  /** @deprecated Do not use, internal ctor. */
+  @Deprecated
+  public HTable(TableName tableName, final HConnection connection,
+      final ExecutorService pool) throws IOException {
+    this(tableName, (ClusterConnection)connection, pool);
+  }
+
   /**
    * Creates an object to access a HBase table.
    * Shares zookeeper connection and other resources with other HTable instances
    * created with the same <code>connection</code> instance.
-   * Use this constructor when the ExecutorService and HConnection instance are
-   * externally managed.
+   * Visible only for HTableWrapper which is in different package.
+   * Should not be used by exernal code.
    * @param tableName Name of the table.
    * @param connection HConnection to be used.
    * @param pool ExecutorService to be used.
    * @throws IOException if a remote or network exception occurs
    */
-  public HTable(TableName tableName, final HConnection connection,
+  @InterfaceAudience.Private
+  public HTable(TableName tableName, final ClusterConnection connection,
       final ExecutorService pool) throws IOException {
     if (connection == null || connection.isClosed()) {
       throw new IllegalArgumentException("Connection is null or closed.");
@@ -506,6 +519,7 @@ public class HTable implements HTableInt
    */
   // TODO(tsuna): Remove this.  Unit tests shouldn't require public helpers.
   @Deprecated
+  @VisibleForTesting
   public HConnection getConnection() {
     return this.connection;
   }

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java Mon Feb 10 04:16:34 2014
@@ -76,9 +76,8 @@ public class MetaScanner {
    * null if not interested in a particular table.
    * @throws IOException e
    */
-  public static void metaScan(Configuration configuration, HConnection connection,
-      MetaScannerVisitor visitor, TableName userTableName)
-  throws IOException {
+  public static void metaScan(Configuration configuration, ClusterConnection connection,
+      MetaScannerVisitor visitor, TableName userTableName) throws IOException {
     metaScan(configuration, connection, visitor, userTableName, null, Integer.MAX_VALUE,
         TableName.META_TABLE_NAME);
   }
@@ -123,10 +122,10 @@ public class MetaScanner {
    * @param metaTableName Meta table to scan, root or meta.
    * @throws IOException e
    */
-  public static void metaScan(Configuration configuration, HConnection connection,
+  static void metaScan(Configuration configuration, ClusterConnection connection,
       final MetaScannerVisitor visitor, final TableName tableName,
       final byte[] row, final int rowLimit, final TableName metaTableName)
-  throws IOException {
+    throws IOException {
     int rowUpperLimit = rowLimit > 0 ? rowLimit: Integer.MAX_VALUE;
     HTable metaTable;
     if (connection == null) {
@@ -266,7 +265,7 @@ public class MetaScanner {
    * @throws IOException
    */
   public static NavigableMap<HRegionInfo, ServerName> allTableRegions(Configuration conf,
-      HConnection connection, final TableName tableName,
+      ClusterConnection connection, final TableName tableName,
       final boolean offlined) throws IOException {
     final NavigableMap<HRegionInfo, ServerName> regions =
       new TreeMap<HRegionInfo, ServerName>();

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java Mon Feb 10 04:16:34 2014
@@ -39,14 +39,14 @@ import java.io.IOException;
 class ZooKeeperKeepAliveConnection extends ZooKeeperWatcher{
   ZooKeeperKeepAliveConnection(
     Configuration conf, String descriptor,
-    HConnectionManager.HConnectionImplementation conn) throws IOException {
+    ConnectionManager.HConnectionImplementation conn) throws IOException {
     super(conf, descriptor, conn);
   }
 
   @Override
   public void close() {
     if (this.abortable != null) {
-      ((HConnectionManager.HConnectionImplementation)abortable).releaseZooKeeperWatcher(this);
+      ((ConnectionManager.HConnectionImplementation)abortable).releaseZooKeeperWatcher(this);
     }
   }
 

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java Mon Feb 10 04:16:34 2014
@@ -38,14 +38,14 @@ import org.apache.zookeeper.KeeperExcept
 class ZooKeeperRegistry implements Registry {
   static final Log LOG = LogFactory.getLog(ZooKeeperRegistry.class);
   // Needs an instance of hci to function.  Set after construct this instance.
-  HConnectionManager.HConnectionImplementation hci;
+  ConnectionManager.HConnectionImplementation hci;
 
   @Override
   public void init(HConnection connection) {
-    if (!(connection instanceof HConnectionManager.HConnectionImplementation)) {
+    if (!(connection instanceof ConnectionManager.HConnectionImplementation)) {
       throw new RuntimeException("This registry depends on HConnectionImplementation");
     }
-    this.hci = (HConnectionManager.HConnectionImplementation)connection;
+    this.hci = (ConnectionManager.HConnectionImplementation)connection;
   }
 
   @Override

Modified: hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java (original)
+++ hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java Mon Feb 10 04:16:34 2014
@@ -118,18 +118,18 @@ public class TestAsyncProcess {
       }
     }
 
-    public MyAsyncProcess(HConnection hc, Configuration conf) {
+    public MyAsyncProcess(ClusterConnection hc, Configuration conf) {
       this(hc, conf, new AtomicInteger());
     }
 
-    public MyAsyncProcess(HConnection hc, Configuration conf, AtomicInteger nbThreads) {
+    public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) {
       super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
           new SynchronousQueue<Runnable>(), new CountingThreadFactory(nbThreads)),
             new RpcRetryingCallerFactory(conf), false);
     }
 
     public MyAsyncProcess(
-          HConnection hc, Configuration conf, boolean useGlobalErrors) {
+        ClusterConnection hc, Configuration conf, boolean useGlobalErrors) {
       super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
         new SynchronousQueue<Runnable>(), new CountingThreadFactory(new AtomicInteger())),
           new RpcRetryingCallerFactory(conf), useGlobalErrors);
@@ -184,7 +184,7 @@ public class TestAsyncProcess {
   /**
    * Returns our async process.
    */
-  static class MyConnectionImpl extends HConnectionManager.HConnectionImplementation {
+  static class MyConnectionImpl extends ConnectionManager.HConnectionImplementation {
     final AtomicInteger nbThreads = new AtomicInteger(0);
     final static Configuration c = new Configuration();
 
@@ -237,7 +237,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testSubmit() throws Exception {
-    HConnection hc = createHConnection();
+    ClusterConnection hc = createHConnection();
     AsyncProcess ap = new MyAsyncProcess(hc, conf);
 
     List<Put> puts = new ArrayList<Put>();
@@ -249,7 +249,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testSubmitWithCB() throws Exception {
-    HConnection hc = createHConnection();
+    ClusterConnection hc = createHConnection();
     final AtomicInteger updateCalled = new AtomicInteger(0);
     Batch.Callback<Object> cb = new Batch.Callback<Object>() {
       public void update(byte[] region, byte[] row, Object result) {
@@ -269,7 +269,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testSubmitBusyRegion() throws Exception {
-    HConnection hc = createHConnection();
+    ClusterConnection hc = createHConnection();
     AsyncProcess ap = new MyAsyncProcess(hc, conf);
 
     List<Put> puts = new ArrayList<Put>();
@@ -287,7 +287,7 @@ public class TestAsyncProcess {
 
   @Test
   public void testSubmitBusyRegionServer() throws Exception {
-    HConnection hc = createHConnection();
+    ClusterConnection hc = createHConnection();
     AsyncProcess ap = new MyAsyncProcess(hc, conf);
 
     ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer));
@@ -462,8 +462,8 @@ public class TestAsyncProcess {
     Assert.assertTrue(start + 100L + sleepTime > end);
   }
 
-  private static HConnection createHConnection() throws IOException {
-    HConnection hc = Mockito.mock(HConnection.class);
+  private static ClusterConnection createHConnection() throws IOException {
+    ClusterConnection hc = Mockito.mock(ClusterConnection.class);
 
     Mockito.when(hc.getRegionLocation(Mockito.eq(DUMMY_TABLE),
         Mockito.eq(DUMMY_BYTES_1), Mockito.anyBoolean())).thenReturn(loc1);
@@ -638,7 +638,7 @@ public class TestAsyncProcess {
   public void testErrorsServers() throws IOException {
     HTable ht = new HTable();
     Configuration configuration = new Configuration(conf);
-    configuration.setBoolean(HConnectionManager.RETRIES_BY_SERVER_KEY, true);
+    configuration.setBoolean(ConnectionManager.RETRIES_BY_SERVER_KEY, true);
     configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
     // set default writeBufferSize
     ht.setWriteBufferSize(configuration.getLong("hbase.client.write.buffer", 2097152));

Modified: hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java (original)
+++ hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java Mon Feb 10 04:16:34 2014
@@ -254,7 +254,7 @@ public class TestClientNoCluster extends
    * Override to shutdown going to zookeeper for cluster id and meta location.
    */
   static class ScanOpenNextThenExceptionThenRecoverConnection
-  extends HConnectionManager.HConnectionImplementation {
+  extends ConnectionManager.HConnectionImplementation {
     final ClientService.BlockingInterface stub;
 
     ScanOpenNextThenExceptionThenRecoverConnection(Configuration conf,
@@ -286,7 +286,7 @@ public class TestClientNoCluster extends
    * Override to shutdown going to zookeeper for cluster id and meta location.
    */
   static class RegionServerStoppedOnScannerOpenConnection
-  extends HConnectionManager.HConnectionImplementation {
+  extends ConnectionManager.HConnectionImplementation {
     final ClientService.BlockingInterface stub;
 
     RegionServerStoppedOnScannerOpenConnection(Configuration conf, boolean managed,
@@ -318,7 +318,7 @@ public class TestClientNoCluster extends
    * Override to check we are setting rpc timeout right.
    */
   static class RpcTimeoutConnection
-  extends HConnectionManager.HConnectionImplementation {
+  extends ConnectionManager.HConnectionImplementation {
     final ClientService.BlockingInterface stub;
 
     RpcTimeoutConnection(Configuration conf, boolean managed, ExecutorService pool, User user)
@@ -345,7 +345,7 @@ public class TestClientNoCluster extends
    * Fake many regionservers and many regions on a connection implementation.
    */
   static class ManyServersManyRegionsConnection
-  extends HConnectionManager.HConnectionImplementation {
+  extends ConnectionManager.HConnectionImplementation {
     // All access should be synchronized
     final Map<ServerName, ClientService.BlockingInterface> serversByClient;
 

Modified: hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java (original)
+++ hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java Mon Feb 10 04:16:34 2014
@@ -71,8 +71,8 @@ public class TestSnapshotFromAdmin {
         + "- further testing won't prove anything.", time < ignoreExpectedTime);
 
     // setup the mocks
-    HConnectionManager.HConnectionImplementation mockConnection = Mockito
-        .mock(HConnectionManager.HConnectionImplementation.class);
+    ConnectionManager.HConnectionImplementation mockConnection = Mockito
+        .mock(ConnectionManager.HConnectionImplementation.class);
     Configuration conf = HBaseConfiguration.create();
     // setup the conf to match the expected properties
     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries);
@@ -118,8 +118,8 @@ public class TestSnapshotFromAdmin {
    */
   @Test
   public void testValidateSnapshotName() throws Exception {
-    HConnectionManager.HConnectionImplementation mockConnection = Mockito
-        .mock(HConnectionManager.HConnectionImplementation.class);
+    ConnectionManager.HConnectionImplementation mockConnection = Mockito
+        .mock(ConnectionManager.HConnectionImplementation.class);
     Configuration conf = HBaseConfiguration.create();
     Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
     HBaseAdmin admin = new HBaseAdmin(mockConnection);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java Mon Feb 10 04:16:34 2014
@@ -62,8 +62,8 @@ import com.google.protobuf.ServiceExcept
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class CoprocessorHConnection implements HConnection {
-  private static final NonceGenerator ng = new HConnectionManager.NoNonceGenerator();
+public class CoprocessorHConnection implements ClusterConnection {
+  private static final NonceGenerator ng = new ConnectionManager.NoNonceGenerator();
 
   /**
    * Create an unmanaged {@link HConnection} based on the environment in which we are running the
@@ -73,9 +73,10 @@ public class CoprocessorHConnection impl
    * @return an unmanaged {@link HConnection}.
    * @throws IOException if we cannot create the basic connection
    */
-  public static HConnection getConnectionForEnvironment(CoprocessorEnvironment env)
+  public static ClusterConnection getConnectionForEnvironment(CoprocessorEnvironment env)
       throws IOException {
-    HConnection connection = HConnectionManager.createConnection(env.getConfiguration());
+    ClusterConnection connection =
+        ConnectionManager.createConnectionInternal(env.getConfiguration());
     // this bit is a little hacky - just trying to get it going for the moment
     if (env instanceof RegionCoprocessorEnvironment) {
       RegionCoprocessorEnvironment e = (RegionCoprocessorEnvironment) env;
@@ -87,11 +88,11 @@ public class CoprocessorHConnection impl
     return connection;
   }
 
-  private HConnection delegate;
+  private ClusterConnection delegate;
   private ServerName serverName;
   private HRegionServer server;
 
-  public CoprocessorHConnection(HConnection delegate, HRegionServer server) {
+  public CoprocessorHConnection(ClusterConnection delegate, HRegionServer server) {
     this.server = server;
     this.serverName = server.getServerName();
     this.delegate = delegate;

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java?rev=1566501&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java Mon Feb 10 04:16:34 2014
@@ -0,0 +1,331 @@
+/**
+ *
+ * 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.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Row;
+import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost.Environment;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.io.MultipleIOException;
+
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+
+/**
+ * A wrapper for HTable. Can be used to restrict privilege.
+ *
+ * Currently it just helps to track tables opened by a Coprocessor and
+ * facilitate close of them if it is aborted.
+ *
+ * We also disallow row locking.
+ *
+ * There is nothing now that will stop a coprocessor from using HTable
+ * objects directly instead of this API, but in the future we intend to
+ * analyze coprocessor implementations as they are loaded and reject those
+ * which attempt to use objects and methods outside the Environment
+ * sandbox.
+ */
+public class HTableWrapper implements HTableInterface {
+
+  private TableName tableName;
+  private HTable table;
+  private ClusterConnection connection;
+  private final List<HTableInterface> openTables;
+
+  /**
+   * @param openTables External list of tables used for tracking wrappers.
+   * @throws IOException 
+   */
+  public static HTableInterface createWrapper(List<HTableInterface> openTables,
+      TableName tableName, Environment env, ExecutorService pool) throws IOException {
+    return new HTableWrapper(openTables, tableName,
+        CoprocessorHConnection.getConnectionForEnvironment(env), pool);
+  }
+
+  private HTableWrapper(List<HTableInterface> openTables, TableName tableName,
+      ClusterConnection connection, ExecutorService pool)
+      throws IOException {
+    this.tableName = tableName;
+    this.table = new HTable(tableName, connection, pool);
+    this.connection = connection;
+    this.openTables = openTables;
+    this.openTables.add(this);
+  }
+
+  public void internalClose() throws IOException {
+    List<IOException> exceptions = new ArrayList<IOException>(2);
+    try {
+    table.close();
+    } catch (IOException e) {
+      exceptions.add(e);
+    }
+    try {
+      // have to self-manage our connection, as per the HTable contract
+      if (this.connection != null) {
+        this.connection.close();
+      }
+    } catch (IOException e) {
+      exceptions.add(e);
+    }
+    if (!exceptions.isEmpty()) {
+      throw MultipleIOException.createIOException(exceptions);
+    }
+  }
+
+  public Configuration getConfiguration() {
+    return table.getConfiguration();
+  }
+
+  public void close() throws IOException {
+    try {
+      internalClose();
+    } finally {
+      openTables.remove(this);
+    }
+  }
+
+  public Result getRowOrBefore(byte[] row, byte[] family)
+      throws IOException {
+    return table.getRowOrBefore(row, family);
+  }
+
+  public Result get(Get get) throws IOException {
+    return table.get(get);
+  }
+
+  public boolean exists(Get get) throws IOException {
+    return table.exists(get);
+  }
+
+  public Boolean[] exists(List<Get> gets) throws IOException{
+    return table.exists(gets);
+  }
+
+  public void put(Put put) throws IOException {
+    table.put(put);
+  }
+
+  public void put(List<Put> puts) throws IOException {
+    table.put(puts);
+  }
+
+  public void delete(Delete delete) throws IOException {
+    table.delete(delete);
+  }
+
+  public void delete(List<Delete> deletes) throws IOException {
+    table.delete(deletes);
+  }
+
+  public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
+      byte[] value, Put put) throws IOException {
+    return table.checkAndPut(row, family, qualifier, value, put);
+  }
+
+  public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
+      CompareOp compareOp, byte[] value, Put put) throws IOException {
+    return table.checkAndPut(row, family, qualifier, compareOp, value, put);
+  }
+
+  public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
+      byte[] value, Delete delete) throws IOException {
+    return table.checkAndDelete(row, family, qualifier, value, delete);
+  }
+
+  public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
+      CompareOp compareOp, byte[] value, Delete delete) throws IOException {
+    return table.checkAndDelete(row, family, qualifier, compareOp, value, delete);
+  }
+
+  public long incrementColumnValue(byte[] row, byte[] family,
+      byte[] qualifier, long amount) throws IOException {
+    return table.incrementColumnValue(row, family, qualifier, amount);
+  }
+
+  public long incrementColumnValue(byte[] row, byte[] family,
+      byte[] qualifier, long amount, Durability durability)
+      throws IOException {
+    return table.incrementColumnValue(row, family, qualifier, amount,
+        durability);
+  }
+
+  @Override
+  public Result append(Append append) throws IOException {
+    return table.append(append);
+  }
+
+  @Override
+  public Result increment(Increment increment) throws IOException {
+    return table.increment(increment);
+  }
+
+  public void flushCommits() throws IOException {
+    table.flushCommits();
+  }
+
+  public boolean isAutoFlush() {
+    return table.isAutoFlush();
+  }
+
+  public ResultScanner getScanner(Scan scan) throws IOException {
+    return table.getScanner(scan);
+  }
+
+  public ResultScanner getScanner(byte[] family) throws IOException {
+    return table.getScanner(family);
+  }
+
+  public ResultScanner getScanner(byte[] family, byte[] qualifier)
+      throws IOException {
+    return table.getScanner(family, qualifier);
+  }
+
+  public HTableDescriptor getTableDescriptor() throws IOException {
+    return table.getTableDescriptor();
+  }
+
+  @Override
+  public byte[] getTableName() {
+    return tableName.getName();
+  }
+
+  @Override
+  public TableName getName() {
+    return table.getName();
+  }
+
+  @Override
+  public void batch(List<? extends Row> actions, Object[] results)
+      throws IOException, InterruptedException {
+    table.batch(actions, results);
+  }
+
+  /**
+   * {@inheritDoc}
+   * @deprecated If any exception is thrown by one of the actions, there is no way to
+   * retrieve the partially executed results. Use {@link #batch(List, Object[])} instead.
+   */
+  @Override
+  public Object[] batch(List<? extends Row> actions)
+      throws IOException, InterruptedException {
+    return table.batch(actions);
+  }
+
+  @Override
+  public <R> void batchCallback(List<? extends Row> actions, Object[] results,
+      Batch.Callback<R> callback) throws IOException, InterruptedException {
+    table.batchCallback(actions, results, callback);
+  }
+
+  /**
+   * {@inheritDoc}
+   * @deprecated If any exception is thrown by one of the actions, there is no way to
+   * retrieve the partially executed results. Use 
+   * {@link #batchCallback(List, Object[], org.apache.hadoop.hbase.client.coprocessor.Batch.Callback)}
+   * instead.
+   */
+  @Override
+  public <R> Object[] batchCallback(List<? extends Row> actions,
+      Batch.Callback<R> callback) throws IOException, InterruptedException {
+    return table.batchCallback(actions, callback);
+  }
+
+  @Override
+  public Result[] get(List<Get> gets) throws IOException {
+    return table.get(gets);
+  }
+
+  @Override
+  public CoprocessorRpcChannel coprocessorService(byte[] row) {
+    return table.coprocessorService(row);
+  }
+
+  @Override
+  public <T extends Service, R> Map<byte[], R> coprocessorService(Class<T> service,
+      byte[] startKey, byte[] endKey, Batch.Call<T, R> callable)
+      throws ServiceException, Throwable {
+    return table.coprocessorService(service, startKey, endKey, callable);
+  }
+
+  @Override
+  public <T extends Service, R> void coprocessorService(Class<T> service,
+      byte[] startKey, byte[] endKey, Batch.Call<T, R> callable, Batch.Callback<R> callback)
+      throws ServiceException, Throwable {
+    table.coprocessorService(service, startKey, endKey, callable, callback);
+  }
+
+  @Override
+  public void mutateRow(RowMutations rm) throws IOException {
+    table.mutateRow(rm);
+  }
+
+  @Override
+  public void setAutoFlush(boolean autoFlush) {
+    table.setAutoFlush(autoFlush, autoFlush);
+  }
+
+  @Override
+  public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
+    table.setAutoFlush(autoFlush, clearBufferOnFail);
+  }
+
+  @Override
+  public void setAutoFlushTo(boolean autoFlush) {
+    table.setAutoFlushTo(autoFlush);
+  }
+
+  @Override
+  public long getWriteBufferSize() {
+     return table.getWriteBufferSize();
+  }
+
+  @Override
+  public void setWriteBufferSize(long writeBufferSize) throws IOException {
+    table.setWriteBufferSize(writeBufferSize);
+  }
+
+  @Override
+  public long incrementColumnValue(byte[] row, byte[] family,
+      byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
+    return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
+  }
+}
\ No newline at end of file

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java Mon Feb 10 04:16:34 2014
@@ -25,7 +25,6 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -44,34 +43,17 @@ import org.apache.hadoop.hbase.Coprocess
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.CoprocessorHConnection;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.hbase.client.HTableWrapper;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
 import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet;
 import org.apache.hadoop.hbase.util.VersionInfo;
-import org.apache.hadoop.io.MultipleIOException;
 
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
 
 /**
  * Provides the common setup framework and runtime services for coprocessor
@@ -357,273 +339,6 @@ public abstract class CoprocessorHost<E 
    */
   public static class Environment implements CoprocessorEnvironment {
 
-    /**
-     * A wrapper for HTable. Can be used to restrict privilege.
-     *
-     * Currently it just helps to track tables opened by a Coprocessor and
-     * facilitate close of them if it is aborted.
-     *
-     * We also disallow row locking.
-     *
-     * There is nothing now that will stop a coprocessor from using HTable
-     * objects directly instead of this API, but in the future we intend to
-     * analyze coprocessor implementations as they are loaded and reject those
-     * which attempt to use objects and methods outside the Environment
-     * sandbox.
-     */
-    class HTableWrapper implements HTableInterface {
-
-      private TableName tableName;
-      private HTable table;
-      private HConnection connection;
-
-      public HTableWrapper(TableName tableName, HConnection connection, ExecutorService pool)
-          throws IOException {
-        this.tableName = tableName;
-        this.table = new HTable(tableName, connection, pool);
-        this.connection = connection;
-        openTables.add(this);
-      }
-
-      void internalClose() throws IOException {
-        List<IOException> exceptions = new ArrayList<IOException>(2);
-        try {
-        table.close();
-        } catch (IOException e) {
-          exceptions.add(e);
-        }
-        try {
-          // have to self-manage our connection, as per the HTable contract
-          if (this.connection != null) {
-            this.connection.close();
-          }
-        } catch (IOException e) {
-          exceptions.add(e);
-        }
-        if (!exceptions.isEmpty()) {
-          throw MultipleIOException.createIOException(exceptions);
-        }
-      }
-
-      public Configuration getConfiguration() {
-        return table.getConfiguration();
-      }
-
-      public void close() throws IOException {
-        try {
-          internalClose();
-        } finally {
-          openTables.remove(this);
-        }
-      }
-
-      public Result getRowOrBefore(byte[] row, byte[] family)
-          throws IOException {
-        return table.getRowOrBefore(row, family);
-      }
-
-      public Result get(Get get) throws IOException {
-        return table.get(get);
-      }
-
-      public boolean exists(Get get) throws IOException {
-        return table.exists(get);
-      }
-
-      public Boolean[] exists(List<Get> gets) throws IOException{
-        return table.exists(gets);
-      }
-
-      public void put(Put put) throws IOException {
-        table.put(put);
-      }
-
-      public void put(List<Put> puts) throws IOException {
-        table.put(puts);
-      }
-
-      public void delete(Delete delete) throws IOException {
-        table.delete(delete);
-      }
-
-      public void delete(List<Delete> deletes) throws IOException {
-        table.delete(deletes);
-      }
-
-      public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-          byte[] value, Put put) throws IOException {
-        return table.checkAndPut(row, family, qualifier, value, put);
-      }
-
-      public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-          CompareOp compareOp, byte[] value, Put put) throws IOException {
-        return table.checkAndPut(row, family, qualifier, compareOp, value, put);
-      }
-
-      public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-          byte[] value, Delete delete) throws IOException {
-        return table.checkAndDelete(row, family, qualifier, value, delete);
-      }
-
-      public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-          CompareOp compareOp, byte[] value, Delete delete) throws IOException {
-        return table.checkAndDelete(row, family, qualifier, compareOp, value, delete);
-      }
-
-      public long incrementColumnValue(byte[] row, byte[] family,
-          byte[] qualifier, long amount) throws IOException {
-        return table.incrementColumnValue(row, family, qualifier, amount);
-      }
-
-      public long incrementColumnValue(byte[] row, byte[] family,
-          byte[] qualifier, long amount, Durability durability)
-          throws IOException {
-        return table.incrementColumnValue(row, family, qualifier, amount,
-            durability);
-      }
-
-      @Override
-      public Result append(Append append) throws IOException {
-        return table.append(append);
-      }
-
-      @Override
-      public Result increment(Increment increment) throws IOException {
-        return table.increment(increment);
-      }
-
-      public void flushCommits() throws IOException {
-        table.flushCommits();
-      }
-
-      public boolean isAutoFlush() {
-        return table.isAutoFlush();
-      }
-
-      public ResultScanner getScanner(Scan scan) throws IOException {
-        return table.getScanner(scan);
-      }
-
-      public ResultScanner getScanner(byte[] family) throws IOException {
-        return table.getScanner(family);
-      }
-
-      public ResultScanner getScanner(byte[] family, byte[] qualifier)
-          throws IOException {
-        return table.getScanner(family, qualifier);
-      }
-
-      public HTableDescriptor getTableDescriptor() throws IOException {
-        return table.getTableDescriptor();
-      }
-
-      @Override
-      public byte[] getTableName() {
-        return tableName.getName();
-      }
-
-      @Override
-      public TableName getName() {
-        return table.getName();
-      }
-
-      @Override
-      public void batch(List<? extends Row> actions, Object[] results)
-          throws IOException, InterruptedException {
-        table.batch(actions, results);
-      }
-
-      /**
-       * {@inheritDoc}
-       * @deprecated If any exception is thrown by one of the actions, there is no way to
-       * retrieve the partially executed results. Use {@link #batch(List, Object[])} instead.
-       */
-      @Override
-      public Object[] batch(List<? extends Row> actions)
-          throws IOException, InterruptedException {
-        return table.batch(actions);
-      }
-
-      @Override
-      public <R> void batchCallback(List<? extends Row> actions, Object[] results,
-          Batch.Callback<R> callback) throws IOException, InterruptedException {
-        table.batchCallback(actions, results, callback);
-      }
-
-      /**
-       * {@inheritDoc}
-       * @deprecated If any exception is thrown by one of the actions, there is no way to
-       * retrieve the partially executed results. Use 
-       * {@link #batchCallback(List, Object[], org.apache.hadoop.hbase.client.coprocessor.Batch.Callback)}
-       * instead.
-       */
-      @Override
-      public <R> Object[] batchCallback(List<? extends Row> actions,
-          Batch.Callback<R> callback) throws IOException, InterruptedException {
-        return table.batchCallback(actions, callback);
-      }
-
-      @Override
-      public Result[] get(List<Get> gets) throws IOException {
-        return table.get(gets);
-      }
-
-      @Override
-      public CoprocessorRpcChannel coprocessorService(byte[] row) {
-        return table.coprocessorService(row);
-      }
-
-      @Override
-      public <T extends Service, R> Map<byte[], R> coprocessorService(Class<T> service,
-          byte[] startKey, byte[] endKey, Batch.Call<T, R> callable)
-          throws ServiceException, Throwable {
-        return table.coprocessorService(service, startKey, endKey, callable);
-      }
-
-      @Override
-      public <T extends Service, R> void coprocessorService(Class<T> service,
-          byte[] startKey, byte[] endKey, Batch.Call<T, R> callable, Batch.Callback<R> callback)
-          throws ServiceException, Throwable {
-        table.coprocessorService(service, startKey, endKey, callable, callback);
-      }
-
-      @Override
-      public void mutateRow(RowMutations rm) throws IOException {
-        table.mutateRow(rm);
-      }
-
-      @Override
-      public void setAutoFlush(boolean autoFlush) {
-        table.setAutoFlush(autoFlush, autoFlush);
-      }
-
-      @Override
-      public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
-        table.setAutoFlush(autoFlush, clearBufferOnFail);
-      }
-
-      @Override
-      public void setAutoFlushTo(boolean autoFlush) {
-        table.setAutoFlushTo(autoFlush);
-      }
-
-      @Override
-      public long getWriteBufferSize() {
-         return table.getWriteBufferSize();
-      }
-
-      @Override
-      public void setWriteBufferSize(long writeBufferSize) throws IOException {
-        table.setWriteBufferSize(writeBufferSize);
-      }
-
-      @Override
-      public long incrementColumnValue(byte[] row, byte[] family,
-          byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
-        return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
-      }
-    }
-
     /** The coprocessor */
     public Coprocessor impl;
     /** Chaining priority */
@@ -757,8 +472,7 @@ public abstract class CoprocessorHost<E 
      */
     @Override
     public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException {
-      return new HTableWrapper(tableName, CoprocessorHConnection.getConnectionForEnvironment(this),
-          pool);
+      return HTableWrapper.createWrapper(openTables, tableName, this, pool);
     }
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Mon Feb 10 04:16:34 2014
@@ -71,7 +71,7 @@ import org.apache.hadoop.hbase.TableNotF
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaReader;
-import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.MetaScanner;
 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
@@ -442,7 +442,7 @@ MasterServices, Server {
     }
     String name = "master/" + initialIsa.toString();
     // Set how many times to retry talking to another server over HConnection.
-    HConnectionManager.setServerSideHConnectionRetries(this.conf, name, LOG);
+    ConnectionUtils.setServerSideHConnectionRetriesConfig(this.conf, name, LOG);
     int numHandlers = conf.getInt(HConstants.MASTER_HANDLER_COUNT,
       conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, HConstants.DEFAULT_MASTER_HANLDER_COUNT));
     this.rpcServer = new RpcServer(this, name, getServices(),

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Mon Feb 10 04:16:34 2014
@@ -84,9 +84,9 @@ import org.apache.hadoop.hbase.catalog.C
 import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
@@ -581,7 +581,7 @@ public class HRegionServer implements Cl
     this.rand = new Random(initialIsa.hashCode());
     String name = "regionserver/" + initialIsa.toString();
     // Set how many times to retry talking to another server over HConnection.
-    HConnectionManager.setServerSideHConnectionRetries(this.conf, name, LOG);
+    ConnectionUtils.setServerSideHConnectionRetriesConfig(this.conf, name, LOG);
     this.priority = new AnnotationReadingPriorityFunction(this);
     RpcSchedulerFactory rpcSchedulerFactory;
     try {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java Mon Feb 10 04:16:34 2014
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.ServerNam
 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.HConnectionManager.HConnectionImplementation;
+import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation;
 import org.mockito.Mockito;
 
 /**
@@ -50,16 +50,16 @@ public class HConnectionTestingUtility {
    * @return HConnection object for <code>conf</code>
    * @throws ZooKeeperConnectionException
    */
-  public static HConnection getMockedConnection(final Configuration conf)
+  public static ClusterConnection getMockedConnection(final Configuration conf)
   throws ZooKeeperConnectionException {
     HConnectionKey connectionKey = new HConnectionKey(conf);
-    synchronized (HConnectionManager.CONNECTION_INSTANCES) {
+    synchronized (ConnectionManager.CONNECTION_INSTANCES) {
       HConnectionImplementation connection =
-        HConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
+          ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
       if (connection == null) {
         connection = Mockito.mock(HConnectionImplementation.class);
         Mockito.when(connection.getConfiguration()).thenReturn(conf);
-        HConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
+        ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
       }
       return connection;
     }
@@ -93,12 +93,12 @@ public class HConnectionTestingUtility {
    * when done with this mocked Connection.
    * @throws IOException
    */
-  public static HConnection getMockedConnectionAndDecorate(final Configuration conf,
+  public static ClusterConnection getMockedConnectionAndDecorate(final Configuration conf,
       final AdminProtos.AdminService.BlockingInterface admin,
       final ClientProtos.ClientService.BlockingInterface client,
       final ServerName sn, final HRegionInfo hri)
   throws IOException {
-    HConnection c = HConnectionTestingUtility.getMockedConnection(conf);
+    ClusterConnection c = HConnectionTestingUtility.getMockedConnection(conf);
     Mockito.doNothing().when(c).close();
     // Make it so we return a particular location when asked.
     final HRegionLocation loc = new HRegionLocation(hri, sn);
@@ -139,12 +139,12 @@ public class HConnectionTestingUtility {
   public static HConnection getSpiedConnection(final Configuration conf)
   throws IOException {
     HConnectionKey connectionKey = new HConnectionKey(conf);
-    synchronized (HConnectionManager.CONNECTION_INSTANCES) {
+    synchronized (ConnectionManager.CONNECTION_INSTANCES) {
       HConnectionImplementation connection =
-        HConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
+          ConnectionManager.CONNECTION_INSTANCES.get(connectionKey);
       if (connection == null) {
         connection = Mockito.spy(new HConnectionImplementation(conf, true));
-        HConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
+        ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
       }
       return connection;
     }
@@ -154,8 +154,8 @@ public class HConnectionTestingUtility {
    * @return Count of extant connection instances
    */
   public static int getConnectionCount() {
-    synchronized (HConnectionManager.CONNECTION_INSTANCES) {
-      return HConnectionManager.CONNECTION_INSTANCES.size();
+    synchronized (ConnectionManager.CONNECTION_INSTANCES) {
+      return ConnectionManager.CONNECTION_INSTANCES.size();
     }
   }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java Mon Feb 10 04:16:34 2014
@@ -102,7 +102,8 @@ public class TestClientTimeouts {
           assertFalse(connection == lastConnection);
           lastConnection = connection;
           // Override the connection's rpc client for timeout testing
-          ((HConnectionManager.HConnectionImplementation)connection).setRpcClient(rpcClient);
+          ((ConnectionManager.HConnectionImplementation)connection).setRpcClient(
+              rpcClient);
           // run some admin commands
           HBaseAdmin.checkHBaseAvailable(conf);
           admin.setBalancerRunning(false, false);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java Mon Feb 10 04:16:34 2014
@@ -241,8 +241,8 @@ public class TestFromClientSide {
      z0.close();
 
      // Then a ZooKeeperKeepAliveConnection
-     HConnectionManager.HConnectionImplementation connection1 =
-       (HConnectionManager.HConnectionImplementation)
+     ConnectionManager.HConnectionImplementation connection1 =
+       (ConnectionManager.HConnectionImplementation)
          HConnectionManager.getConnection(newConfig);
 
      ZooKeeperKeepAliveConnection z1 = connection1.getKeepAliveZooKeeperWatcher();
@@ -263,8 +263,8 @@ public class TestFromClientSide {
 
      Configuration newConfig2 = new Configuration(TEST_UTIL.getConfiguration());
      newConfig2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "6789");
-     HConnectionManager.HConnectionImplementation connection2 =
-       (HConnectionManager.HConnectionImplementation)
+     ConnectionManager.HConnectionImplementation connection2 =
+       (ConnectionManager.HConnectionImplementation)
          HConnectionManager.getConnection(newConfig2);
 
      assertTrue("connections should be different ", connection1 != connection2);
@@ -275,7 +275,7 @@ public class TestFromClientSide {
          " on different connections", z1 != z3);
 
      // Bypass the private access
-     Method m = HConnectionManager.HConnectionImplementation.class.
+     Method m = ConnectionManager.HConnectionImplementation.class.
        getDeclaredMethod("closeZooKeeperWatcher");
      m.setAccessible(true);
      m.invoke(connection2);
@@ -338,7 +338,7 @@ public class TestFromClientSide {
     TEST_UTIL.countRows(table);
     table.getConnection().clearRegionCache();
     assertEquals("Clearing cache should have 0 cached ", 0,
-        HConnectionManager.getCachedRegionCount(conf, TABLENAME));
+        ConnectionManager.getCachedRegionCount(conf, TABLENAME));
 
     // A Get is suppose to do a region lookup request
     Get g = new Get(Bytes.toBytes("aaa"));
@@ -346,7 +346,7 @@ public class TestFromClientSide {
 
     // only one region should be cached if the cache prefetch is disabled.
     assertEquals("Number of cached region is incorrect ", 1,
-        HConnectionManager.getCachedRegionCount(conf, TABLENAME));
+        ConnectionManager.getCachedRegionCount(conf, TABLENAME));
 
     // now we enable cached prefetch.
     HTable.setRegionCachePrefetch(conf, TABLENAME, true);
@@ -364,7 +364,7 @@ public class TestFromClientSide {
     table.getConnection().clearRegionCache();
 
     assertEquals("Number of cached region is incorrect ", 0,
-        HConnectionManager.getCachedRegionCount(conf, TABLENAME));
+        ConnectionManager.getCachedRegionCount(conf, TABLENAME));
 
     // if there is a cache miss, some additional regions should be prefetched.
     Get g2 = new Get(Bytes.toBytes("bbb"));
@@ -376,14 +376,14 @@ public class TestFromClientSide {
     // the total number of cached regions == region('aaa") + prefeched regions.
     LOG.info("Testing how many regions cached");
     assertEquals("Number of cached region is incorrect ", prefetchRegionNumber,
-        HConnectionManager.getCachedRegionCount(conf, TABLENAME));
+        ConnectionManager.getCachedRegionCount(conf, TABLENAME));
 
     table.getConnection().clearRegionCache();
 
     Get g3 = new Get(Bytes.toBytes("abc"));
     table.get(g3);
     assertEquals("Number of cached region is incorrect ", prefetchRegionNumber,
-        HConnectionManager.getCachedRegionCount(conf, TABLENAME));
+        ConnectionManager.getCachedRegionCount(conf, TABLENAME));
 
     LOG.info("Finishing testRegionCachePreWarm");
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java Mon Feb 10 04:16:34 2014
@@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.MediumTes
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation;
+import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -320,9 +320,9 @@ public class TestHCM {
     // Save off current HConnections
     Map<HConnectionKey, HConnectionImplementation> oldHBaseInstances =
         new HashMap<HConnectionKey, HConnectionImplementation>();
-    oldHBaseInstances.putAll(HConnectionManager.CONNECTION_INSTANCES);
+    oldHBaseInstances.putAll(ConnectionManager.CONNECTION_INSTANCES);
 
-    HConnectionManager.CONNECTION_INSTANCES.clear();
+    ConnectionManager.CONNECTION_INSTANCES.clear();
 
     try {
       HConnection connection = HConnectionManager.getConnection(TEST_UTIL.getConfiguration());
@@ -332,8 +332,8 @@ public class TestHCM {
         HConnectionManager.getConnection(TEST_UTIL.getConfiguration()));
     } finally {
       // Put original HConnections back
-      HConnectionManager.CONNECTION_INSTANCES.clear();
-      HConnectionManager.CONNECTION_INSTANCES.putAll(oldHBaseInstances);
+      ConnectionManager.CONNECTION_INSTANCES.clear();
+      ConnectionManager.CONNECTION_INSTANCES.putAll(oldHBaseInstances);
     }
   }
 
@@ -354,8 +354,8 @@ public class TestHCM {
     Put put = new Put(ROW);
     put.add(FAM_NAM, ROW, ROW);
     table.put(put);
-    HConnectionManager.HConnectionImplementation conn =
-      (HConnectionManager.HConnectionImplementation)table.getConnection();
+    ConnectionManager.HConnectionImplementation conn =
+      (ConnectionManager.HConnectionImplementation)table.getConnection();
 
     assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
 
@@ -455,7 +455,7 @@ public class TestHCM {
       Assert.assertArrayEquals(e.getRow(0).getRow(), ROW);
 
       // Check that we unserialized the exception as expected
-      Throwable cause = HConnectionManager.findException(e.getCause(0));
+      Throwable cause = ConnectionManager.findException(e.getCause(0));
       Assert.assertNotNull(cause);
       Assert.assertTrue(cause instanceof RegionMovedException);
     }
@@ -548,8 +548,8 @@ public class TestHCM {
     Put put = new Put(ROW);
     put.add(FAM_NAM, ROW, ROW);
     table.put(put);
-    HConnectionManager.HConnectionImplementation conn =
-      (HConnectionManager.HConnectionImplementation)table.getConnection();
+    ConnectionManager.HConnectionImplementation conn =
+      (ConnectionManager.HConnectionImplementation)table.getConnection();
 
     HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW);
     assertNotNull(location);
@@ -616,7 +616,7 @@ public class TestHCM {
 
   /**
    * Makes sure that there is no leaking of
-   * {@link HConnectionManager.HConnectionImplementation} in the {@link HConnectionManager}
+   * {@link ConnectionManager.HConnectionImplementation} in the {@link HConnectionManager}
    * class.
    */
   @Test
@@ -756,8 +756,8 @@ public class TestHCM {
   public void testMulti() throws Exception {
     HTable table = TEST_UTIL.createTable(TABLE_NAME3, FAM_NAM);
     TEST_UTIL.createMultiRegions(table, FAM_NAM);
-    HConnectionManager.HConnectionImplementation conn =
-      (HConnectionManager.HConnectionImplementation)
+    ConnectionManager.HConnectionImplementation conn =
+      (ConnectionManager.HConnectionImplementation)
         HConnectionManager.getConnection(TEST_UTIL.getConfiguration());
 
     // We're now going to move the region and check that it works for the client
@@ -875,8 +875,8 @@ public class TestHCM {
     try {
       long timeBase = timeMachine.currentTimeMillis();
       long largeAmountOfTime = ANY_PAUSE * 1000;
-      HConnectionManager.ServerErrorTracker tracker =
-          new HConnectionManager.ServerErrorTracker(largeAmountOfTime, 100);
+      ConnectionManager.ServerErrorTracker tracker =
+          new ConnectionManager.ServerErrorTracker(largeAmountOfTime, 100);
 
       // The default backoff is 0.
       assertEquals(0, tracker.calculateBackoffTime(location, ANY_PAUSE));
@@ -976,9 +976,9 @@ public class TestHCM {
     for (int i = 0; i < 30; i++) {
       HConnection c1 = null;
       try {
-        c1 = HConnectionManager.getConnection(config);
+        c1 = ConnectionManager.getConnectionInternal(config);
         LOG.info("HTable connection " + i + " " + c1);
-        HTable table = new HTable(TABLE_NAME4, c1, pool);
+        HTable table = new HTable(config, TABLE_NAME4, pool);
         table.close();
         LOG.info("HTable connection " + i + " closed " + c1);
       } catch (Exception e) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java Mon Feb 10 04:16:34 2014
@@ -507,7 +507,7 @@ public class TestMultiParallel {
       }
     };
     NonceGenerator oldCnm =
-        HConnectionManager.injectNonceGeneratorForTesting(table.getConnection(), cnm);
+        ConnectionUtils.injectNonceGeneratorForTesting(table.getConnection(), cnm);
 
     // First test sequential requests.
     try {
@@ -570,7 +570,7 @@ public class TestMultiParallel {
       validateResult(result, QUALIFIER, Bytes.toBytes((numRequests / 2) + 1L));
       table.close();
     } finally {
-      HConnectionManager.injectNonceGeneratorForTesting(table.getConnection(), oldCnm);
+      ConnectionManager.injectNonceGeneratorForTesting(table.getConnection(), oldCnm);
     }
   }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java Mon Feb 10 04:16:34 2014
@@ -57,7 +57,7 @@ import org.junit.experimental.categories
 import static org.junit.Assert.*;
 
 /**
- * Tests class {@link org.apache.hadoop.hbase.coprocessor.CoprocessorHost.Environment.HTableWrapper}
+ * Tests class {@link org.apache.hadoop.hbase.client.HTableWrapper}
  * by invoking its methods and briefly asserting the result is reasonable.
  */
 @Category(MediumTests.class)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java Mon Feb 10 04:16:34 2014
@@ -71,7 +71,7 @@ import org.apache.hadoop.hbase.SplitLogC
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.NonceGenerator;
@@ -322,7 +322,7 @@ public class TestDistributedLogSplitting
     HTable ht = installTable(zkw, TABLE_NAME, FAMILY_NAME, NUM_REGIONS_TO_CREATE);
     NonceGeneratorWithDups ng = new NonceGeneratorWithDups();
     NonceGenerator oldNg =
-        HConnectionManager.injectNonceGeneratorForTesting(ht.getConnection(), ng);
+        ConnectionUtils.injectNonceGeneratorForTesting(ht.getConnection(), ng);
 
     try {
       List<Increment> reqs = new ArrayList<Increment>();
@@ -356,7 +356,7 @@ public class TestDistributedLogSplitting
         }
       }
     } finally {
-      HConnectionManager.injectNonceGeneratorForTesting(ht.getConnection(), oldNg);
+      ConnectionUtils.injectNonceGeneratorForTesting(ht.getConnection(), oldNg);
       ht.close();
       zkw.close();
     }