Package org.apache.hadoop.hbase.client


package org.apache.hadoop.hbase.client
Provides HBase Client

Table of Contents

Overview

To administer HBase, create and drop tables, list and alter tables, use Admin. Once created, table access is via an instance of Table. You add content to a table a row at a time. To insert, create an instance of a Put object. Specify value, target column and optionally a timestamp. Commit your update using Table.put(Put). To fetch your inserted value, use Get. The Get can be specified to be broad -- get all on a particular row -- or narrow; i.e. return only a single cell value. After creating an instance of Get, invoke Table.get(Get).

Use Scan to set up a scanner -- a Cursor- like access. After creating and configuring your Scan instance, call Table.getScanner(Scan) and then invoke next on the returned object. Both Table.get(Get) and Table.getScanner(Scan) return a Result.

Use Delete to remove content. You can remove individual cells or entire families, etc. Pass it to Table.delete(Delete) to execute.

Puts, Gets and Deletes take out a lock on the target row for the duration of their operation. Concurrent modifications to a single row are serialized. Gets and scans run concurrently without interference of the row locks and are guaranteed to not to return half written rows.

Client code accessing a cluster finds the cluster by querying ZooKeeper. This means that the ZooKeeper quorum to use must be on the client CLASSPATH. Usually this means make sure the client can find your hbase-site.xml.

Example API Usage

Once you have a running HBase, you probably want a way to hook your application up to it. If your application is in Java, then you should use the Java API. Here's an example of what a simple client might look like. This example assumes that you've created a table called "myTable" with a column family called "myColumnFamily".

import java.io.IOException;

import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Table;
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.Scan;
import org.apache.hadoop.hbase.util.Bytes;

// Class that has nothing but a main.
// Does a Put, Get and a Scan against an hbase table.
// The API described here is since HBase 1.0.
public class MyLittleHBaseClient {
  public static void main(String[] args) throws IOException {
    // You need a configuration object to tell the client where to connect.
    // When you create a HBaseConfiguration, it reads in whatever you've set
    // into your hbase-site.xml and in hbase-default.xml, as long as these can
    // be found on the CLASSPATH
    Configuration config = HBaseConfiguration.create();

    // Next you need a Connection to the cluster. Create one. When done with it,
    // close it. A try/finally is a good way to ensure it gets closed or use
    // the jdk7 idiom, try-with-resources: see
    // https://docs.oracle.com/javase/tutorial/essential/exceptions/tryResourceClose.html
    //
    // Connections are heavyweight.  Create one once and keep it around. From a Connection
    // you get a Table instance to access Tables, an Admin instance to administer the cluster,
    // and RegionLocator to find where regions are out on the cluster. As opposed to Connections,
    // Table, Admin and RegionLocator instances are lightweight; create as you need them and then
    // close when done.
    //
    Connection connection = ConnectionFactory.createConnection(config);
    try {

      // The below instantiates a Table object that connects you to the "myLittleHBaseTable" table
      // (TableName.valueOf turns String into a TableName instance).
      // When done with it, close it (Should start a try/finally after this creation so it gets
      // closed for sure the jdk7 idiom, try-with-resources: see
      // https://docs.oracle.com/javase/tutorial/essential/exceptions/tryResourceClose.html)
      Table table = connection.getTable(TableName.valueOf("myLittleHBaseTable"));
      try {

        // To add to a row, use Put.  A Put constructor takes the name of the row
        // you want to insert into as a byte array.  In HBase, the Bytes class has
        // utility for converting all kinds of java types to byte arrays.  In the
        // below, we are converting the String "myLittleRow" into a byte array to
        // use as a row key for our update. Once you have a Put instance, you can
        // adorn it by setting the names of columns you want to update on the row,
        // the timestamp to use in your update, etc. If no timestamp, the server
        // applies current time to the edits.
        Put p = new Put(Bytes.toBytes("myLittleRow"));

        // To set the value you'd like to update in the row 'myLittleRow', specify
        // the column family, column qualifier, and value of the table cell you'd
        // like to update.  The column family must already exist in your table
        // schema.  The qualifier can be anything.  All must be specified as byte
        // arrays as hbase is all about byte arrays.  Lets pretend the table
        // 'myLittleHBaseTable' was created with a family 'myLittleFamily'.
        p.add(Bytes.toBytes("myLittleFamily"), Bytes.toBytes("someQualifier"),
        Bytes.toBytes("Some Value"));

        // Once you've adorned your Put instance with all the updates you want to
        // make, to commit it do the following (The HTable#put method takes the
        // Put instance you've been building and pushes the changes you made into
        // hbase)
        table.put(p);

        // Now, to retrieve the data we just wrote. The values that come back are
        // Result instances. Generally, a Result is an object that will package up
        // the hbase return into the form you find most palatable.
        Get g = new Get(Bytes.toBytes("myLittleRow"));
        Result r = table.get(g);
        byte [] value = r.getValue(Bytes.toBytes("myLittleFamily"),
          Bytes.toBytes("someQualifier"));

        // If we convert the value bytes, we should get back 'Some Value', the
        // value we inserted at this location.
        String valueStr = Bytes.toString(value);
        System.out.println("GET: " + valueStr);

        // Sometimes, you won't know the row you're looking for. In this case, you
        // use a Scanner. This will give you cursor-like interface to the contents
        // of the table.  To set up a Scanner, do like you did above making a Put
        // and a Get, create a Scan.  Adorn it with column names, etc.
        Scan s = new Scan();
        s.addColumn(Bytes.toBytes("myLittleFamily"), Bytes.toBytes("someQualifier"));
        ResultScanner scanner = table.getScanner(s);
        try {
           // Scanners return Result instances.
           // Now, for the actual iteration. One way is to use a while loop like so:
           for (Result rr = scanner.next(); rr != null; rr = scanner.next()) {
             // print out the row we found and the columns we were looking for
             System.out.println("Found row: " + rr);
           }

           // The other approach is to use a foreach loop. Scanners are iterable!
           // for (Result rr : scanner) {
           //   System.out.println("Found row: " + rr);
           // }
         } finally {
           // Make sure you close your scanners when you are done!
           // Thats why we have it inside a try/finally clause
           scanner.close();
         }

         // Close your table and cluster connection.
       } finally {
         if (table != null) table.close();
       }
     } finally {
       connection.close();
     }
  }
}

There are many other methods for putting data into and getting data out of HBase, but these examples should get you started. See the Table javadoc for more methods. Additionally, there are methods for managing tables in the Admin class.

If your client is NOT Java, then you should consider the Thrift or REST libraries.

Related Documentation

See also the section in the HBase Reference Guide where it discusses HBase Client. It has section on how to access HBase from inside your multithreaded environment how to control resources consumed client-side, etc.

  • Class
    Description
    org.apache.hadoop.hbase.client.AbstractClientScanner
    Helper class for custom client scanners.
    org.apache.hadoop.hbase.client.Action
    A Get, Put, Increment, Append, or Delete associated with it's region.
    org.apache.hadoop.hbase.client.Admin
    The administrative API for HBase.
    org.apache.hadoop.hbase.client.AdvancedScanResultConsumer
    This is the low level API for asynchronous scan.
    org.apache.hadoop.hbase.client.AdvancedScanResultConsumer.ScanController
    Used to suspend or stop a scan, or get a scan cursor if available.
    org.apache.hadoop.hbase.client.AdvancedScanResultConsumer.ScanResumer
    Used to resume a scan.
    org.apache.hadoop.hbase.client.Append
    Performs Append operations on a single row.
    org.apache.hadoop.hbase.client.AsyncAdmin
    The asynchronous administrative API for HBase.
    org.apache.hadoop.hbase.client.AsyncAdminBuilder
    For creating AsyncAdmin.
    org.apache.hadoop.hbase.client.AsyncAdminClientUtils
    Additional Asynchronous Admin capabilities for clients.
    org.apache.hadoop.hbase.client.AsyncAdminRequestRetryingCaller<T>
     
    org.apache.hadoop.hbase.client.AsyncAdminRequestRetryingCaller.Callable<T>
     
    org.apache.hadoop.hbase.client.AsyncBufferedMutator
    Used to communicate with a single HBase table in batches.
    org.apache.hadoop.hbase.client.AsyncBufferedMutatorBuilder
    For creating AsyncBufferedMutator.
    org.apache.hadoop.hbase.client.AsyncConnection
    The asynchronous version of Connection.
    org.apache.hadoop.hbase.client.AsyncConnectionImpl
    The implementation of AsyncConnection.
    org.apache.hadoop.hbase.client.AsyncMasterRequestRpcRetryingCaller<T>
    Retry caller for a request call to master.
    org.apache.hadoop.hbase.client.AsyncMasterRequestRpcRetryingCaller.Callable<T>
     
    org.apache.hadoop.hbase.client.AsyncProcessTask<T>
    Contains the attributes of a task which will be executed by AsyncProcess.
    org.apache.hadoop.hbase.client.AsyncProcessTask.Builder<T>
     
    org.apache.hadoop.hbase.client.AsyncProcessTask.SubmittedRows
    The number of processed rows.
    org.apache.hadoop.hbase.client.AsyncRequestFuture
    The context used to wait for results from one submit call.
    org.apache.hadoop.hbase.client.AsyncRpcRetryingCaller<T>
     
    org.apache.hadoop.hbase.client.AsyncServerRequestRpcRetryingCaller<T>
    Retry caller for a request call to region server.
    org.apache.hadoop.hbase.client.AsyncServerRequestRpcRetryingCaller.Callable<T>
     
    org.apache.hadoop.hbase.client.AsyncTable<C extends org.apache.hadoop.hbase.client.ScanResultConsumerBase>
    The interface for asynchronous version of Table.
    org.apache.hadoop.hbase.client.AsyncTable.CheckAndMutateBuilder
    Deprecated.
    Since 2.4.0, will be removed in 4.0.0.
    org.apache.hadoop.hbase.client.AsyncTable.CheckAndMutateWithFilterBuilder
    Deprecated.
    Since 2.4.0, will be removed in 4.0.0.
    org.apache.hadoop.hbase.client.AsyncTable.CoprocessorCallback<R>
    The callback when we want to execute a coprocessor call on a range of regions.
    org.apache.hadoop.hbase.client.AsyncTable.CoprocessorServiceBuilder<S,R>
    Helper class for sending coprocessorService request that executes a coprocessor call on regions which are covered by a range.
    org.apache.hadoop.hbase.client.AsyncTableBuilder<C extends org.apache.hadoop.hbase.client.ScanResultConsumerBase>
    For creating AsyncTable.
    org.apache.hadoop.hbase.client.AsyncTableRegionLocator
    The asynchronous version of RegionLocator.
    org.apache.hadoop.hbase.client.Attributes
     
    History of balancer decisions taken for region movements.
    org.apache.hadoop.hbase.client.BalancerDecision.Builder
     
    org.apache.hadoop.hbase.client.BalanceRequest
    Encapsulates options for executing a run of the Balancer.
    org.apache.hadoop.hbase.client.BalanceRequest.Builder
    Builder for constructing a BalanceRequest
    org.apache.hadoop.hbase.client.BalanceResponse
    Response returned from a balancer invocation
    org.apache.hadoop.hbase.client.BalanceResponse.Builder
    Used in HMaster to build a BalanceResponse for returning results of a balance invocation to callers
    History of detail information that balancer movements was rejected
    org.apache.hadoop.hbase.client.BalancerRejection.Builder
     
    org.apache.hadoop.hbase.client.BatchScanResultCache
    A scan result cache for batched scan, i.e, scan.getBatch() > 0 && !scan.getAllowPartialResults().
    org.apache.hadoop.hbase.client.BufferedMutator
    Used to communicate with a single HBase table similar to Table but meant for batched, asynchronous puts.
    org.apache.hadoop.hbase.client.BufferedMutator.ExceptionListener
    Listens for asynchronous exceptions on a BufferedMutator.
    org.apache.hadoop.hbase.client.BufferedMutatorImpl
    Used to communicate with a single HBase table similar to Table but meant for batched, potentially asynchronous puts.
    org.apache.hadoop.hbase.client.BufferedMutatorParams
    Parameters for instantiating a BufferedMutator.
    Used to perform CheckAndMutate operations.
    A builder class for building a CheckAndMutate object.
    org.apache.hadoop.hbase.client.CheckAndMutateResult
    Represents a result of a CheckAndMutate operation
    org.apache.hadoop.hbase.client.ClientAsyncPrefetchScanner
    ClientAsyncPrefetchScanner implements async scanner behaviour.
    org.apache.hadoop.hbase.client.ClientCoprocessorRpcController
    Client side rpc controller for coprocessor implementation.
    org.apache.hadoop.hbase.client.ClientScanner
    Implements the scanner interface for the HBase client.
    org.apache.hadoop.hbase.client.ClientServiceCallable<T>
    A RegionServerCallable set to use the Client protocol.
    org.apache.hadoop.hbase.client.ClientSideRegionScanner
    A client scanner for a region opened for read-only on the client side.
    org.apache.hadoop.hbase.client.ClientSimpleScanner
    ClientSimpleScanner implements a sync scanner behaviour.
    org.apache.hadoop.hbase.client.ClientUtil
     
    org.apache.hadoop.hbase.client.ClusterConnection
    Internal methods on Connection that should not be used by user code.
    org.apache.hadoop.hbase.client.ColumnFamilyDescriptor
    An ColumnFamilyDescriptor contains information about a column family such as the number of versions, compression settings, etc.
    org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder
     
    org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor
    An ModifyableFamilyDescriptor contains information about a column family such as the number of versions, compression settings, etc.
    org.apache.hadoop.hbase.client.CompactionState
    POJO representing the compaction state
    org.apache.hadoop.hbase.client.CompactType
    Currently, there are only two compact types: NORMAL means do store files compaction; MOB means do mob files compaction.
    org.apache.hadoop.hbase.client.Connection
    A cluster connection encapsulating lower level individual connections to actual servers and a connection to zookeeper.
    org.apache.hadoop.hbase.client.ConnectionConfiguration
    Configuration parameters for the connection.
    org.apache.hadoop.hbase.client.ConnectionFactory
    A non-instantiable class that manages creation of Connections.
    org.apache.hadoop.hbase.client.ConnectionImplementation
    Main implementation of Connection and ClusterConnection interfaces.
    org.apache.hadoop.hbase.client.ConnectionRegistry
    Registry for meta information needed for connection setup to a HBase cluster.
    org.apache.hadoop.hbase.client.ConnectionUtils
    Utility used by client connections.
    org.apache.hadoop.hbase.client.Consistency
    Consistency defines the expected consistency level for an operation.
    org.apache.hadoop.hbase.client.CoprocessorDescriptor
    CoprocessorDescriptor contains the details about how to build a coprocessor.
    org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder
    Used to build the CoprocessorDescriptor
    org.apache.hadoop.hbase.client.Cursor
    Scan cursor to tell client where server is scanning Scan.setNeedCursorResult(boolean) Result.isCursor() Result.getCursor()
    org.apache.hadoop.hbase.client.DelayingRunner
    A wrapper for a runnable for a group of actions for a single regionserver.
    org.apache.hadoop.hbase.client.Delete
    Used to perform Delete operations on a single row.
    org.apache.hadoop.hbase.client.DoNotRetryRegionException
    Similar to RegionException, but disables retries.
    org.apache.hadoop.hbase.client.Durability
    Enum describing the durability guarantees for tables and Mutations Note that the items must be sorted in order of increasing durability
    org.apache.hadoop.hbase.client.FlushRegionCallable
    A Callable for flushRegion() RPC.
    org.apache.hadoop.hbase.client.Get
    Used to perform Get operations on a single row.
    org.apache.hadoop.hbase.client.HBaseAdmin
    HBaseAdmin is no longer a client API.
    org.apache.hadoop.hbase.client.HBaseAdmin.NamespaceFuture
     
    org.apache.hadoop.hbase.client.HBaseAdmin.ProcedureFuture<V>
    Future that waits on a procedure result.
    org.apache.hadoop.hbase.client.HBaseAdmin.ProcedureFuture.WaitForStateCallable
     
    org.apache.hadoop.hbase.client.HBaseAdmin.TableFuture<V>
     
    org.apache.hadoop.hbase.client.HBaseHbck
    Use Connection.getHbck() to obtain an instance of Hbck instead of constructing an HBaseHbck directly.
    org.apache.hadoop.hbase.client.Hbck
    Hbck fixup tool APIs.
    org.apache.hadoop.hbase.client.HRegionLocator
    An implementation of RegionLocator.
    org.apache.hadoop.hbase.client.HTable
    An implementation of Table.
    org.apache.hadoop.hbase.client.HTableMultiplexer
    Deprecated.
    since 2.2.0, will be removed in 3.0.0, without replacement.
    org.apache.hadoop.hbase.client.HTableMultiplexer.HTableMultiplexerStatus
    Deprecated.
    since 2.2.0, will be removed in 3.0.0, without replacement.
    org.apache.hadoop.hbase.client.ImmutableHColumnDescriptor
    Deprecated.
    org.apache.hadoop.hbase.client.ImmutableHRegionInfo
    Deprecated.
    org.apache.hadoop.hbase.client.ImmutableHTableDescriptor
    Deprecated.
    org.apache.hadoop.hbase.client.ImmutableScan
    Immutable version of Scan
    org.apache.hadoop.hbase.client.Increment
    Used to perform Increment operations on a single row.
    org.apache.hadoop.hbase.client.IsolationLevel
    Specify Isolation levels in Scan operations.
    org.apache.hadoop.hbase.client.LockTimeoutException
     
    Abstract response class representing online logs response from ring-buffer use-cases e.g slow/large RPC logs, balancer decision logs
    Deprecated.
    as of 2.4.0.
    org.apache.hadoop.hbase.client.LogQueryFilter.FilterByOperator
     
    org.apache.hadoop.hbase.client.LogQueryFilter.Type
     
    org.apache.hadoop.hbase.client.MasterRegistry
    Deprecated.
    Since 2.5.0, will be removed in 4.0.0.
    org.apache.hadoop.hbase.client.MasterSwitchType
    Represents the master switch type
    org.apache.hadoop.hbase.client.MetaCache
    A cache implementation for region locations from meta.
    org.apache.hadoop.hbase.client.MetricsConnection
    This class is for maintaining the various connection statistics and publishing them through the metrics interfaces.
    org.apache.hadoop.hbase.client.MetricsConnection.CallStats
    A container class for collecting details about the RPC call as it percolates.
    org.apache.hadoop.hbase.client.MetricsConnection.CallTracker
     
    org.apache.hadoop.hbase.client.MetricsConnection.RegionStats
     
    org.apache.hadoop.hbase.client.MetricsConnection.RunnerStats
     
    org.apache.hadoop.hbase.client.MobCompactPartitionPolicy
    Enum describing the mob compact partition policy types.
    org.apache.hadoop.hbase.client.MultiAction
    Container for Actions (i.e.
    org.apache.hadoop.hbase.client.MultiResponse
    A container for Result objects, grouped by regionName.
    org.apache.hadoop.hbase.client.Mutation
     
    org.apache.hadoop.hbase.client.NoncedRegionServerCallable<T>
    Implementations make an rpc call against a RegionService via a protobuf Service.
    org.apache.hadoop.hbase.client.NonceGenerator
    NonceGenerator interface.
    org.apache.hadoop.hbase.client.NormalizeTableFilterParams
    A collection of criteria used for table selection.
    org.apache.hadoop.hbase.client.NormalizeTableFilterParams.Builder
    Used to instantiate an instance of NormalizeTableFilterParams.
    org.apache.hadoop.hbase.client.NoServerForRegionException
    Thrown when no region server can be found for a region
    Slow/Large Log payload for hbase-client, to be used by Admin API get_slow_responses and get_large_responses
    org.apache.hadoop.hbase.client.OnlineLogRecord.OnlineLogRecordBuilder
     
    org.apache.hadoop.hbase.client.Operation
    Superclass for any type that maps to a potentially application-level query.
    org.apache.hadoop.hbase.client.OperationTimeoutExceededException
    Thrown when a batch operation exceeds the operation timeout
    org.apache.hadoop.hbase.client.OperationWithAttributes
     
    org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor
    A helper class used to access the package private field in o.a.h.h.client package.
    org.apache.hadoop.hbase.client.PerClientRandomNonceGenerator
    NonceGenerator implementation that uses client ID hash + random int as nonce group, and random numbers as nonces.
    org.apache.hadoop.hbase.client.Put
    Used to perform Put operations for a single row.
    org.apache.hadoop.hbase.client.Query
    Base class for HBase read operations; e.g.
    org.apache.hadoop.hbase.client.RegionAdminServiceCallable<T>
    Similar to RegionServerCallable but for the AdminService interface.
    org.apache.hadoop.hbase.client.RegionCoprocessorServiceExec
    Represents a coprocessor service method execution against a single region.
    org.apache.hadoop.hbase.client.RegionInfo
    Information about a region.
    org.apache.hadoop.hbase.client.RegionInfoBuilder
     
    org.apache.hadoop.hbase.client.RegionInfoDisplay
    Utility used composing RegionInfo for 'display'; e.g.
    org.apache.hadoop.hbase.client.RegionLoadStats
    POJO representing region server load
    org.apache.hadoop.hbase.client.RegionLocator
    Used to view region location information for a single HBase table.
    org.apache.hadoop.hbase.client.RegionOfflineException
    Thrown when a table can not be located
    org.apache.hadoop.hbase.client.RegionReplicaUtil
    Utility methods which contain the logic for regions and replicas.
    org.apache.hadoop.hbase.client.RegionServerCallable<T,S>
    Implementations make a RPC call against a RegionService via a protobuf Service.
    org.apache.hadoop.hbase.client.RegionServerCoprocessorRpcChannelImpl
    The implementation of a region server based coprocessor rpc channel.
    org.apache.hadoop.hbase.client.RegionServerRegistry
    Connection registry implementation for region server.
    org.apache.hadoop.hbase.client.RegionStatesCount
     
    org.apache.hadoop.hbase.client.RegionStatesCount.RegionStatesCountBuilder
     
    org.apache.hadoop.hbase.client.RequestController
    An interface for client request scheduling algorithm.
    org.apache.hadoop.hbase.client.RequestController.Checker
    Picks up the valid data.
    org.apache.hadoop.hbase.client.RequestController.ReturnCode
     
    org.apache.hadoop.hbase.client.RequestControllerFactory
    A factory class that constructs an RequestController.
    org.apache.hadoop.hbase.client.Result
    Single row result of a Get or Scan query.
    org.apache.hadoop.hbase.client.ResultBoundedCompletionService<V>
    A completion service for the RpcRetryingCallerFactory.
    org.apache.hadoop.hbase.client.ResultScanner
    Interface for client-side scanning.
    org.apache.hadoop.hbase.client.ResultStatsUtil
    Statistics update about a server/region
    org.apache.hadoop.hbase.client.RetriesExhaustedException
    Exception thrown by HTable methods when an attempt to do something (like commit changes) fails after a bunch of retries.
    org.apache.hadoop.hbase.client.RetriesExhaustedException.ThrowableWithExtraContext
    Data structure that allows adding more info around Throwable incident.
    org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException
    This subclass of RetriesExhaustedException is thrown when we have more information about which rows were causing which exceptions on what servers.
    org.apache.hadoop.hbase.client.RetryingCallable<T>
    A Callable<T> that will be retried.
    org.apache.hadoop.hbase.client.ReversedClientScanner
    A reversed client scanner which support backward scanning
    org.apache.hadoop.hbase.client.ReversedScannerCallable
    A reversed ScannerCallable which supports backward scanning.
    org.apache.hadoop.hbase.client.Row
    Has a row.
    org.apache.hadoop.hbase.client.RowAccess<T>
    Provide a way to access the inner buffer.
    org.apache.hadoop.hbase.client.RowMutations
    Performs multiple mutations atomically on a single row.
    org.apache.hadoop.hbase.client.RowTooBigException
    Gets or Scans throw this exception if running without in-row scan flag set and row size appears to exceed max configured size (configurable via hbase.table.max.rowsize).
    org.apache.hadoop.hbase.client.RpcConnectionRegistry
    Rpc based connection registry.
    org.apache.hadoop.hbase.client.RpcRetryingCaller<T>
     
    org.apache.hadoop.hbase.client.RpcRetryingCallerFactory
    Factory to create an RpcRetryingCaller
    org.apache.hadoop.hbase.client.RpcRetryingCallerImpl<T>
    Runs an rpc'ing RetryingCallable.
    org.apache.hadoop.hbase.client.RpcRetryingCallerWithReadReplicas
    Caller that goes to replica if the primary region does no answer within a configurable timeout.
    org.apache.hadoop.hbase.client.Scan
    Used to perform Scan operations.
    org.apache.hadoop.hbase.client.Scan.ReadType
     
    org.apache.hadoop.hbase.client.ScannerCallable
    Scanner operations such as create, next, etc.
    org.apache.hadoop.hbase.client.ScanResultConsumer
    Receives Result for an asynchronous scan.
    org.apache.hadoop.hbase.client.ScanResultConsumerBase
    The base interface for scan result consumer.
    org.apache.hadoop.hbase.client.SecureBulkLoadClient
    Client proxy for SecureBulkLoadProtocol
    org.apache.hadoop.hbase.client.ServerConnectionUtils
     
    org.apache.hadoop.hbase.client.ServerConnectionUtils.ShortCircuitingClusterConnection
    A ClusterConnection that will short-circuit RPC making direct invocations against the localhost if the invocation target is 'this' server; save on network and protobuf invocations.
    org.apache.hadoop.hbase.client.ServerStatisticTracker
    Tracks the statistics for multiple regions
    org.apache.hadoop.hbase.client.ServerType
    Select server type i.e destination for RPC request associated with ring buffer.
    org.apache.hadoop.hbase.client.ServiceCaller<S,R>
    Delegate to a protobuf rpc call.
    org.apache.hadoop.hbase.client.ShortCircuitMasterConnection
    A short-circuit connection that can bypass the RPC layer (serialization, deserialization, networking, etc..) when talking to a local master
    org.apache.hadoop.hbase.client.SingleResponse
    Class for single action response
    org.apache.hadoop.hbase.client.SingleResponse.Entry
     
    org.apache.hadoop.hbase.client.SlowLogParams
    SlowLog params object that contains detailed info as params and region name : to be used for filter purpose
    org.apache.hadoop.hbase.client.SnapshotDescription
    The POJO equivalent of HBaseProtos.SnapshotDescription
    org.apache.hadoop.hbase.client.SnapshotType
    POJO representing the snapshot type
    org.apache.hadoop.hbase.client.StatisticTrackable
    Parent interface for an object to get updates about per-region statistics.
    org.apache.hadoop.hbase.client.Table
    Used to communicate with a single HBase table.
    org.apache.hadoop.hbase.client.Table.CheckAndMutateBuilder
    Deprecated.
    Since 2.4.0, will be removed in 4.0.0.
    org.apache.hadoop.hbase.client.Table.CheckAndMutateWithFilterBuilder
    Deprecated.
    Since 2.4.0, will be removed in 4.0.0.
    org.apache.hadoop.hbase.client.TableBuilder
    For creating Table instance.
    org.apache.hadoop.hbase.client.TableDescriptor
    TableDescriptor contains the details about an HBase table such as the descriptors of all the column families, is the table a catalog table, hbase:meta , if the table is read only, the maximum size of the memstore, when the region split should occur, coprocessors associated with it etc...
    org.apache.hadoop.hbase.client.TableDescriptorBuilder
    Convenience class for composing an instance of TableDescriptor.
    org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor
    TODO: make this private after removing the HTableDescriptor
    org.apache.hadoop.hbase.client.TableDescriptorUtils
     
    org.apache.hadoop.hbase.client.TableDescriptorUtils.TableDescriptorDelta
     
    org.apache.hadoop.hbase.client.TableSnapshotScanner
    A Scanner which performs a scan over snapshot files.
    org.apache.hadoop.hbase.client.TableState
    Represents table state.
    org.apache.hadoop.hbase.client.TableState.State
     
    org.apache.hadoop.hbase.client.VersionInfoUtil
    Class to help with parsing the version info.
    org.apache.hadoop.hbase.client.WrongRowIOException