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 [3/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/ConnectionUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java Mon Feb 10 04:16:34 2014
@@ -19,7 +19,9 @@ package org.apache.hadoop.hbase.client;
 
 import java.util.Random;
 
+import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 
 /**
@@ -61,4 +63,33 @@ public class ConnectionUtils {
     }
     return newPause;
   }
+  
+  /**
+   * @param conn The connection for which to replace the generator.
+   * @param cnm Replaces the nonce generator used, for testing.
+   * @return old nonce generator.
+   */
+  public static NonceGenerator injectNonceGeneratorForTesting(
+      HConnection conn, NonceGenerator cnm) {
+    return ConnectionManager.injectNonceGeneratorForTesting(conn, cnm);
+  }
+
+  /**
+   * Changes the configuration to set the number of retries needed when using HConnection
+   * internally, e.g. for  updating catalog tables, etc.
+   * Call this method before we create any Connections.
+   * @param c The Configuration instance to set the retries into.
+   * @param log Used to log what we set in here.
+   */
+  public static void setServerSideHConnectionRetriesConfig(
+      final Configuration c, final String sn, final Log log) {
+    int hcRetries = c.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+      HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
+    // Go big.  Multiply by 10.  If we can't get to meta after this many retries
+    // then something seriously wrong.
+    int serversideMultiplier = c.getInt("hbase.client.serverside.retries.multiplier", 10);
+    int retries = hcRetries * serversideMultiplier;
+    c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
+    log.debug(sn + " HConnection server-to-server retries=" + retries);
+  }
 }

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Mon Feb 10 04:16:34 2014
@@ -163,7 +163,7 @@ public class HBaseAdmin implements Abort
 
   // We use the implementation class rather then the interface because we
   //  need the package protected functions to get the connection to master
-  private HConnection connection;
+  private ClusterConnection connection;
 
   private volatile Configuration conf;
   private final long pause;
@@ -188,20 +188,26 @@ public class HBaseAdmin implements Abort
   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
     // Will not leak connections, as the new implementation of the constructor
     // does not throw exceptions anymore.
-    this(HConnectionManager.getConnection(new Configuration(c)));
+    this(ConnectionManager.getConnectionInternal(new Configuration(c)));
     this.cleanupConnectionOnClose = true;
   }
 
- /**
-  * Constructor for externally managed HConnections.
-  * The connection to master will be created when required by admin functions.
-  *
-  * @param connection The HConnection instance to use
-  * @throws MasterNotRunningException, ZooKeeperConnectionException are not
-  *  thrown anymore but kept into the interface for backward api compatibility
-  */
+  /**
+   * Constructor for externally managed HConnections.
+   * The connection to master will be created when required by admin functions.
+   *
+   * @param connection The HConnection instance to use
+   * @throws MasterNotRunningException, ZooKeeperConnectionException are not
+   *  thrown anymore but kept into the interface for backward api compatibility
+   * @deprecated Do not use this internal ctor.
+   */
+  @Deprecated
   public HBaseAdmin(HConnection connection)
       throws MasterNotRunningException, ZooKeeperConnectionException {
+    this((ClusterConnection)connection);
+  }
+
+  HBaseAdmin(ClusterConnection connection) {
     this.conf = connection.getConfiguration();
     this.connection = connection;
 
@@ -2324,8 +2330,8 @@ public class HBaseAdmin implements Abort
     copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
     copyOfConf.setInt("zookeeper.recovery.retry", 0);
 
-    HConnectionManager.HConnectionImplementation connection
-      = (HConnectionManager.HConnectionImplementation)
+    ConnectionManager.HConnectionImplementation connection
+      = (ConnectionManager.HConnectionImplementation)
       HConnectionManager.getConnection(copyOfConf);
 
     try {

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java Mon Feb 10 04:16:34 2014
@@ -154,7 +154,9 @@ public interface HConnection extends Abo
    */
   public HTableInterface getTable(TableName tableName, ExecutorService pool)  throws IOException;
 
-  /** @return - true if the master server is running */
+  /** @return - true if the master server is running
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   boolean isMasterRunning()
   throws MasterNotRunningException, ZooKeeperConnectionException;
 
@@ -202,7 +204,8 @@ public interface HConnection extends Abo
    *          splitKeys used while creating table
    * @throws IOException
    *           if a remote or network exception occurs
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws
       IOException;
 
@@ -251,7 +254,8 @@ public interface HConnection extends Abo
    * @return HRegionLocation that describes where to find the region in
    * question
    * @throws IOException if a remote or network exception occurs
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   public HRegionLocation locateRegion(final TableName tableName,
       final byte [] row) throws IOException;
 
@@ -261,7 +265,8 @@ public interface HConnection extends Abo
 
   /**
    * Allows flushing the region cache.
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   void clearRegionCache();
 
   /**
@@ -269,7 +274,8 @@ public interface HConnection extends Abo
    * <code>tableName</code>
    * @param tableName Name of the table whose regions we are to remove from
    * cache.
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   void clearRegionCache(final TableName tableName);
 
   @Deprecated
@@ -278,7 +284,8 @@ public interface HConnection extends Abo
   /**
    * Deletes cached locations for the specific region.
    * @param location The location object for the region, to be purged from cache.
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   void deleteCachedRegionLocation(final HRegionLocation location);
 
   /**
@@ -289,7 +296,8 @@ public interface HConnection extends Abo
    * @return HRegionLocation that describes where to find the region in
    * question
    * @throws IOException if a remote or network exception occurs
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   HRegionLocation relocateRegion(final TableName tableName,
       final byte [] row) throws IOException;
 
@@ -308,7 +316,8 @@ public interface HConnection extends Abo
    * @param rowkey the row
    * @param exception the exception if any. Can be null.
    * @param source the previous location
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   void updateCachedLocations(TableName tableName, byte[] rowkey,
                                     Object exception, ServerName source);
 
@@ -322,7 +331,8 @@ public interface HConnection extends Abo
    * @return HRegionLocation that describes where to find the region in
    * question
    * @throws IOException if a remote or network exception occurs
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   HRegionLocation locateRegion(final byte[] regionName)
   throws IOException;
 
@@ -331,7 +341,8 @@ public interface HConnection extends Abo
    * @param tableName table to get regions of
    * @return list of region locations for all regions of table
    * @throws IOException
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   List<HRegionLocation> locateRegions(final TableName tableName) throws IOException;
 
   @Deprecated
@@ -345,7 +356,8 @@ public interface HConnection extends Abo
    *          regions from returned list.
    * @return list of region locations for all regions of table
    * @throws IOException
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   public List<HRegionLocation> locateRegions(final TableName tableName,
       final boolean useCache,
       final boolean offlined) throws IOException;
@@ -357,7 +369,8 @@ public interface HConnection extends Abo
 
   /**
    * Returns a {@link MasterKeepAliveConnection} to the active master
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   MasterService.BlockingInterface getMaster() throws IOException;
 
 
@@ -366,7 +379,8 @@ public interface HConnection extends Abo
    * @param serverName
    * @return proxy for HRegionServer
    * @throws IOException if a remote or network exception occurs
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
 
   /**
@@ -376,8 +390,8 @@ public interface HConnection extends Abo
    * @param serverName
    * @return ClientProtocol proxy for RegionServer
    * @throws IOException if a remote or network exception occurs
-   *
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
 
   /**
@@ -398,7 +412,8 @@ public interface HConnection extends Abo
    * @param reload If true do not use cache, otherwise bypass.
    * @return Location of row.
    * @throws IOException if a remote or network exception occurs
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   HRegionLocation getRegionLocation(TableName tableName, byte [] row,
     boolean reload)
   throws IOException;
@@ -460,6 +475,7 @@ public interface HConnection extends Abo
   public void setRegionCachePrefetch(final TableName tableName,
       final boolean enable);
 
+  @Deprecated
   public void setRegionCachePrefetch(final byte[] tableName,
       final boolean enable);
 
@@ -471,6 +487,7 @@ public interface HConnection extends Abo
    */
   boolean getRegionCachePrefetch(final TableName tableName);
 
+  @Deprecated
   boolean getRegionCachePrefetch(final byte[] tableName);
 
   /**
@@ -500,7 +517,8 @@ public interface HConnection extends Abo
   /**
    * Clear any caches that pertain to server name <code>sn</code>.
    * @param sn A server name
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   void clearCaches(final ServerName sn);
 
   /**
@@ -518,16 +536,13 @@ public interface HConnection extends Abo
   /**
    * @param serverName
    * @return true if the server is known as dead, false otherwise.
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   boolean isDeadServer(ServerName serverName);
 
   /**
    * @return Nonce generator for this HConnection; may be null if disabled in configuration.
-   */
+   * @deprecated internal method, do not use thru HConnection */
+  @Deprecated
   public NonceGenerator getNonceGenerator();
-
-  /**
-   * @return Default AsyncProcess associated with this connection.
-   */
-  AsyncProcess getAsyncProcess();
 }
\ No newline at end of file

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java Mon Feb 10 04:16:34 2014
@@ -72,7 +72,8 @@ class HConnectionKey {
         username = currentUser.getName();
       }
     } catch (IOException ioe) {
-      HConnectionManager.LOG.warn("Error obtaining current user, skipping username in HConnectionKey", ioe);
+      ConnectionManager.LOG.warn(
+          "Error obtaining current user, skipping username in HConnectionKey", ioe);
     }
   }