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 [1/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...

Author: sershe
Date: Mon Feb 10 04:16:34 2014
New Revision: 1566501

URL: http://svn.apache.org/r1566501
Log:
HBASE-10479 HConnection interface is public but is used internally, and contains a bunch of methods

Added:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
    hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
    hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
    hbase/trunk/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java?rev=1566501&r1=1566500&r2=1566501&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java Mon Feb 10 04:16:34 2014
@@ -122,7 +122,7 @@ class AsyncProcess {
   // TODO: many of the fields should be made private
   protected final long id;
 
-  protected final HConnection hConnection;
+  protected final ClusterConnection hConnection;
   protected final RpcRetryingCallerFactory rpcCallerFactory;
   protected final BatchErrors globalErrors;
   protected final ExecutorService pool;
@@ -190,7 +190,7 @@ class AsyncProcess {
     }
   }
 
-  public AsyncProcess(HConnection hc, Configuration conf, ExecutorService pool,
+  public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService pool,
       RpcRetryingCallerFactory rpcCaller, boolean useGlobalErrors) {
     if (hc == null) {
       throw new IllegalArgumentException("HConnection cannot be null.");
@@ -508,7 +508,7 @@ class AsyncProcess {
   protected class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
     private final Batch.Callback<CResult> callback;
     private final BatchErrors errors;
-    private final HConnectionManager.ServerErrorTracker errorsByServer;
+    private final ConnectionManager.ServerErrorTracker errorsByServer;
     private final ExecutorService pool;
 
 
@@ -1070,7 +1070,8 @@ class AsyncProcess {
    * We may benefit from connection-wide tracking of server errors.
    * @return ServerErrorTracker to use, null if there is no ServerErrorTracker on this connection
    */
-  protected HConnectionManager.ServerErrorTracker createServerErrorTracker() {
-    return new HConnectionManager.ServerErrorTracker(this.serverTrackerTimeout, this.numTries);
+  protected ConnectionManager.ServerErrorTracker createServerErrorTracker() {
+    return new ConnectionManager.ServerErrorTracker(
+        this.serverTrackerTimeout, this.numTries);
   }
 }

Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java?rev=1566501&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java Mon Feb 10 04:16:34 2014
@@ -0,0 +1,214 @@
+/**
+ *
+ * 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.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+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.MasterProtos.MasterService;
+
+/** Internal methods on HConnection that should not be used by user code. */
+@InterfaceAudience.Private
+// NOTE: DO NOT make this class public. It was made package-private on purpose.
+interface ClusterConnection extends HConnection {
+
+  /** @return - true if the master server is running */
+  boolean isMasterRunning()
+      throws MasterNotRunningException, ZooKeeperConnectionException;
+
+  /**
+   * Use this api to check if the table has been created with the specified number of
+   * splitkeys which was used while creating the given table.
+   * Note : If this api is used after a table's region gets splitted, the api may return
+   * false.
+   * @param tableName
+   *          tableName
+   * @param splitKeys
+   *          splitKeys used while creating table
+   * @throws IOException
+   *           if a remote or network exception occurs
+   */
+  boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws
+      IOException;
+  
+  /**
+   * Find the location of the region of <i>tableName</i> that <i>row</i>
+   * lives in.
+   * @param tableName name of the table <i>row</i> is in
+   * @param row row key you're trying to find the region of
+   * @return HRegionLocation that describes where to find the region in
+   * question
+   * @throws IOException if a remote or network exception occurs
+   */
+  public HRegionLocation locateRegion(final TableName tableName,
+      final byte [] row) throws IOException;
+
+  /**
+   * Allows flushing the region cache.
+   */
+  void clearRegionCache();
+
+  /**
+   * Allows flushing the region cache of all locations that pertain to
+   * <code>tableName</code>
+   * @param tableName Name of the table whose regions we are to remove from
+   * cache.
+   */
+  void clearRegionCache(final TableName tableName);
+
+  /**
+   * Deletes cached locations for the specific region.
+   * @param location The location object for the region, to be purged from cache.
+   */
+  void deleteCachedRegionLocation(final HRegionLocation location);
+
+  /**
+   * Find the location of the region of <i>tableName</i> that <i>row</i>
+   * lives in, ignoring any value that might be in the cache.
+   * @param tableName name of the table <i>row</i> is in
+   * @param row row key you're trying to find the region of
+   * @return HRegionLocation that describes where to find the region in
+   * question
+   * @throws IOException if a remote or network exception occurs
+   */
+  HRegionLocation relocateRegion(final TableName tableName,
+      final byte [] row) throws IOException;
+
+  /**
+   * Update the location cache. This is used internally by HBase, in most cases it should not be
+   *  used by the client application.
+   * @param tableName the table name
+   * @param rowkey the row
+   * @param exception the exception if any. Can be null.
+   * @param source the previous location
+   */
+  void updateCachedLocations(TableName tableName, byte[] rowkey,
+                                    Object exception, ServerName source);
+
+
+  /**
+   * Gets the location of the region of <i>regionName</i>.
+   * @param regionName name of the region to locate
+   * @return HRegionLocation that describes where to find the region in
+   * question
+   * @throws IOException if a remote or network exception occurs
+   */
+  HRegionLocation locateRegion(final byte[] regionName)
+  throws IOException;
+
+  /**
+   * Gets the locations of all regions in the specified table, <i>tableName</i>.
+   * @param tableName table to get regions of
+   * @return list of region locations for all regions of table
+   * @throws IOException
+   */
+  List<HRegionLocation> locateRegions(final TableName tableName) throws IOException;
+
+  /**
+   * Gets the locations of all regions in the specified table, <i>tableName</i>.
+   * @param tableName table to get regions of
+   * @param useCache Should we use the cache to retrieve the region information.
+   * @param offlined True if we are to include offlined regions, false and we'll leave out offlined
+   *          regions from returned list.
+   * @return list of region locations for all regions of table
+   * @throws IOException
+   */
+  List<HRegionLocation> locateRegions(final TableName tableName,
+      final boolean useCache,
+      final boolean offlined) throws IOException;
+
+
+  /**
+   * Returns a {@link MasterKeepAliveConnection} to the active master
+   */
+  MasterService.BlockingInterface getMaster() throws IOException;
+
+
+  /**
+   * Establishes a connection to the region server at the specified address.
+   * @param serverName
+   * @return proxy for HRegionServer
+   * @throws IOException if a remote or network exception occurs
+   */
+  AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
+
+  /**
+   * Establishes a connection to the region server at the specified address, and returns
+   * a region client protocol.
+   *
+   * @param serverName
+   * @return ClientProtocol proxy for RegionServer
+   * @throws IOException if a remote or network exception occurs
+   *
+   */
+  ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
+
+  /**
+   * Find region location hosting passed row
+   * @param tableName table name
+   * @param row Row to find.
+   * @param reload If true do not use cache, otherwise bypass.
+   * @return Location of row.
+   * @throws IOException if a remote or network exception occurs
+   */
+  HRegionLocation getRegionLocation(TableName tableName, byte [] row,
+    boolean reload)
+  throws IOException;
+
+  /**
+   * Clear any caches that pertain to server name <code>sn</code>.
+   * @param sn A server name
+   */
+  void clearCaches(final ServerName sn);
+
+  /**
+   * This function allows HBaseAdmin and potentially others to get a shared MasterService
+   * connection.
+   * @return The shared instance. Never returns null.
+   * @throws MasterNotRunningException
+   */
+  @Deprecated
+  MasterKeepAliveConnection getKeepAliveMasterService()
+  throws MasterNotRunningException;
+
+  /**
+   * @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.
+   */
+  public NonceGenerator getNonceGenerator();
+
+  /**
+   * @return Default AsyncProcess associated with this connection.
+   */
+  AsyncProcess getAsyncProcess();
+}
\ No newline at end of file