Interface | Description |
---|---|
Admin |
The administrative API for HBase.
|
AdvancedScanResultConsumer |
This is the low level API for asynchronous scan.
|
AdvancedScanResultConsumer.ScanController |
Used to suspend or stop a scan, or get a scan cursor if available.
|
AdvancedScanResultConsumer.ScanResumer |
Used to resume a scan.
|
AsyncAdmin |
The asynchronous administrative API for HBase.
|
AsyncAdminBuilder |
For creating
AsyncAdmin . |
AsyncAdminRequestRetryingCaller.Callable<T> | |
AsyncBufferedMutator |
Used to communicate with a single HBase table in batches.
|
AsyncBufferedMutatorBuilder |
For creating
AsyncBufferedMutator . |
AsyncConnection |
The asynchronous version of Connection.
|
AsyncMasterRequestRpcRetryingCaller.Callable<T> | |
AsyncRegistry |
Implementations hold cluster information such as this cluster's id, location of hbase:meta, etc..
|
AsyncRequestFuture |
The context used to wait for results from one submit call.
|
AsyncServerRequestRpcRetryingCaller.Callable<T> | |
AsyncSingleRequestRpcRetryingCaller.Callable<T> | |
AsyncTable<C extends ScanResultConsumerBase> |
The interface for asynchronous version of Table.
|
AsyncTable.CheckAndMutateBuilder |
A helper class for sending checkAndMutate request.
|
AsyncTable.CoprocessorCallback<R> |
The callback when we want to execute a coprocessor call on a range of regions.
|
AsyncTable.CoprocessorServiceBuilder<S,R> |
Helper class for sending coprocessorService request that executes a coprocessor call on regions
which are covered by a range.
|
AsyncTableBuilder<C extends ScanResultConsumerBase> |
For creating
AsyncTable . |
AsyncTableRegionLocator |
The asynchronous version of RegionLocator.
|
Attributes | |
BufferedMutator |
Used to communicate with a single HBase table similar to
Table but meant for
batched, asynchronous puts. |
BufferedMutator.ExceptionListener |
Listens for asynchronous exceptions on a
BufferedMutator . |
Cancellable |
This should be implemented by the Get/Scan implementations that
talk to replica regions.
|
ClusterConnection |
Internal methods on Connection that should not be used by user code.
|
ClusterStatusListener.DeadServerHandler |
Class to be extended to manage a new dead server.
|
ClusterStatusListener.Listener |
The interface to be implemented by a listener of a cluster status event.
|
ColumnFamilyDescriptor |
An ColumnFamilyDescriptor contains information about a column family such as the
number of versions, compression settings, etc.
|
Connection |
A cluster connection encapsulating lower level individual connections to actual servers and
a connection to zookeeper.
|
CoprocessorDescriptor |
CoprocessorDescriptor contains the details about how to build a coprocessor.
|
HBaseAdmin.ProcedureFuture.WaitForStateCallable | |
Hbck |
Hbck APIs for HBase.
|
MasterKeepAliveConnection |
A KeepAlive connection is not physically closed immediately after the close,
but rather kept alive for a few minutes.
|
MetricsConnection.NewMetric<T> |
A lambda for dispatching to the appropriate metric factory method
|
NonceGenerator |
NonceGenerator interface.
|
RawAsyncHBaseAdmin.AdminRpcCall<RESP,REQ> | |
RawAsyncHBaseAdmin.Converter<D,S> | |
RawAsyncHBaseAdmin.MasterRpcCall<RESP,REQ> | |
RawAsyncHBaseAdmin.TableOperator | |
RawAsyncTableImpl.Converter<D,I,S> | |
RawAsyncTableImpl.NoncedConverter<D,I,S> | |
RawAsyncTableImpl.RpcCall<RESP,REQ> | |
RegionInfo |
Information about a region.
|
RegionLocator |
Used to view region location information for a single HBase table.
|
RequestController |
An interface for client request scheduling algorithm.
|
RequestController.Checker |
Picks up the valid data.
|
ResultScanner |
Interface for client-side scanning.
|
RetryingCallable<T> |
A Callable<T> that will be retried.
|
Row |
Has a row.
|
RowAccess<T> |
Provide a way to access the inner buffer.
|
RpcRetryingCaller<T> | |
ScanResultCache |
Used to separate the row constructing logic.
|
ScanResultConsumer |
Receives
Result for an asynchronous scan. |
ScanResultConsumerBase |
The base interface for scan result consumer.
|
ServiceCaller<S,R> |
Delegate to a protobuf rpc call.
|
SimpleRequestController.RowChecker |
Provide a way to control the flow of rows iteration.
|
StatisticTrackable |
Parent interface for an object to get updates about per-region statistics.
|
Table |
Used to communicate with a single HBase table.
|
Table.CheckAndMutateBuilder |
A helper class for sending checkAndMutate request.
|
TableBuilder |
For creating
Table instance. |
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... |
ZKAsyncRegistry.Converter<T> |
Class | Description |
---|---|
AbstractClientScanner |
Helper class for custom client scanners.
|
AbstractResponse |
This class is used to extend AP to process single action request, like delete, get etc.
|
Action |
A Get, Put, Increment, Append, or Delete associated with it's region.
|
AllowPartialScanResultCache |
A ScanResultCache that may return partial result.
|
Append |
Performs Append operations on a single row.
|
AsyncAdminBuilderBase |
Base class for all asynchronous admin builders.
|
AsyncAdminRequestRetryingCaller<T> | |
AsyncBatchRpcRetryingCaller<T> |
Retry caller for batch.
|
AsyncBatchRpcRetryingCaller.RegionRequest | |
AsyncBatchRpcRetryingCaller.ServerRequest | |
AsyncBufferedMutatorBuilderImpl |
The implementation of
AsyncBufferedMutatorBuilder . |
AsyncBufferedMutatorImpl |
The implementation of
AsyncBufferedMutator . |
AsyncClientScanner |
The asynchronous client scanner implementation.
|
AsyncClientScanner.OpenScannerResponse | |
AsyncConnectionConfiguration |
Timeout configs.
|
AsyncConnectionImpl |
The implementation of AsyncConnection.
|
AsyncHBaseAdmin |
Just a wrapper of
RawAsyncHBaseAdmin . |
AsyncMasterRequestRpcRetryingCaller<T> |
Retry caller for a request call to master.
|
AsyncMetaRegionLocator |
The asynchronous locator for meta region.
|
AsyncNonMetaRegionLocator |
The asynchronous locator for regions other than meta.
|
AsyncNonMetaRegionLocator.LocateRequest | |
AsyncNonMetaRegionLocator.TableCache | |
AsyncProcess |
This class allows a continuous flow of requests.
|
AsyncProcessTask<T> |
Contains the attributes of a task which will be executed
by
AsyncProcess . |
AsyncProcessTask.Builder<T> | |
AsyncProcessTask.ListRowAccess<T> | |
AsyncRegionLocator |
The asynchronous region locator.
|
AsyncRegionLocatorHelper |
Helper class for asynchronous region locator.
|
AsyncRegistryFactory |
Get instance of configured Registry.
|
AsyncRequestFutureImpl<CResult> |
The context, and return value, for a single submit/submitAll call.
|
AsyncRequestFutureImpl.ReplicaResultState |
Sync point for calls to multiple replicas for the same user request (Get).
|
AsyncRpcRetryingCaller<T> | |
AsyncRpcRetryingCallerFactory |
Factory to create an AsyncRpcRetryCaller.
|
AsyncScanSingleRegionRpcRetryingCaller |
Retry caller for scanning a region.
|
AsyncServerRequestRpcRetryingCaller<T> |
Retry caller for a request call to region server.
|
AsyncSingleRequestRpcRetryingCaller<T> |
Retry caller for a single request, such as get, put, delete, etc.
|
AsyncTableBuilderBase<C extends ScanResultConsumerBase> |
Base class for all asynchronous table builders.
|
AsyncTableImpl |
Just a wrapper of
RawAsyncTableImpl . |
AsyncTableRegionLocatorImpl |
The implementation of AsyncRegionLocator.
|
AsyncTableResultScanner |
The
ResultScanner implementation for AsyncTable . |
BatchErrors | |
BatchScanResultCache |
A scan result cache for batched scan, i.e,
scan.getBatch() > 0 && !scan.getAllowPartialResults() . |
BufferedMutatorImpl |
Used to communicate with a single HBase table similar to
Table
but meant for batched, potentially asynchronous puts. |
BufferedMutatorParams |
Parameters for instantiating a
BufferedMutator . |
CancellableRegionServerCallable<T> |
This class is used to unify HTable calls with AsyncProcess Framework.
|
ClientAsyncPrefetchScanner |
ClientAsyncPrefetchScanner implements async scanner behaviour.
|
ClientCoprocessorRpcController |
Client side rpc controller for coprocessor implementation.
|
ClientIdGenerator |
The class that is able to determine some unique strings for the client,
such as an IP address, PID, and composite deterministic ID.
|
ClientScanner |
Implements the scanner interface for the HBase client.
|
ClientServiceCallable<T> |
A RegionServerCallable set to use the Client protocol.
|
ClientSideRegionScanner |
A client scanner for a region opened for read-only on the client side.
|
ClientSimpleScanner |
ClientSimpleScanner implements a sync scanner behaviour.
|
ClientUtil | |
ClusterStatusListener |
A class that receives the cluster status, and provide it as a set of service to the client.
|
ColumnFamilyDescriptorBuilder | |
ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor |
An ModifyableFamilyDescriptor contains information about a column family such as the
number of versions, compression settings, etc.
|
CompleteScanResultCache |
A scan result cache that only returns complete result.
|
ConnectionConfiguration |
Configuration parameters for the connection.
|
ConnectionFactory |
A non-instantiable class that manages creation of
Connection s. |
ConnectionImplementation |
Main implementation of
Connection and ClusterConnection interfaces. |
ConnectionImplementation.MasterServiceState |
State of the MasterService connection/setup.
|
ConnectionImplementation.ServerErrorTracker |
The record of errors for servers.
|
ConnectionImplementation.ServerErrorTracker.ServerErrors |
The record of errors for a server.
|
ConnectionUtils |
Utility used by client connections.
|
ConnectionUtils.MasterlessConnection |
Some tests shut down the master.
|
ConnectionUtils.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.
|
CoprocessorDescriptorBuilder |
Used to build the
CoprocessorDescriptor |
CoprocessorDescriptorBuilder.CoprocessorDescriptorImpl | |
Cursor |
Scan cursor to tell client where server is scanning
Scan.setNeedCursorResult(boolean)
Result.isCursor()
Result.getCursor() |
DelayingRunner |
A wrapper for a runnable for a group of actions for a single regionserver.
|
Delete |
Used to perform Delete operations on a single row.
|
FailureInfo |
Keeps track of repeated failures to any region server.
|
FastFailInterceptorContext | |
FlushRegionCallable |
A Callable for flushRegion() RPC.
|
Get |
Used to perform Get operations on a single row.
|
HBaseAdmin |
HBaseAdmin is no longer a client API.
|
HBaseAdmin.AbortProcedureFuture | |
HBaseAdmin.AddColumnFamilyFuture | |
HBaseAdmin.CreateTableFuture | |
HBaseAdmin.DeleteColumnFamilyFuture | |
HBaseAdmin.DeleteTableFuture | |
HBaseAdmin.DisableTableFuture | |
HBaseAdmin.EnableTableFuture | |
HBaseAdmin.MergeTableRegionsFuture | |
HBaseAdmin.ModifyColumnFamilyFuture | |
HBaseAdmin.ModifyTableFuture | |
HBaseAdmin.NamespaceFuture | |
HBaseAdmin.ProcedureFuture<V> |
Future that waits on a procedure result.
|
HBaseAdmin.RestoreSnapshotFuture | |
HBaseAdmin.SplitTableRegionFuture | |
HBaseAdmin.TableFuture<V> | |
HBaseAdmin.ThrowableAbortable |
Simple
Abortable , throwing RuntimeException on abort. |
HBaseAdmin.TruncateTableFuture | |
HBaseHbck |
Use
ClusterConnection.getHbck() to obtain an instance of Hbck instead of
constructing
an HBaseHbck directly. |
HRegionLocator |
An implementation of
RegionLocator . |
HTable |
An implementation of
Table . |
HTableMultiplexer |
HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
|
HTableMultiplexer.AtomicAverageCounter |
Helper to count the average over an interval until reset.
|
HTableMultiplexer.FlushWorker | |
HTableMultiplexer.HTableMultiplexerStatus |
HTableMultiplexerStatus keeps track of the current status of the HTableMultiplexer.
|
HTableMultiplexer.PutStatus | |
ImmutableHColumnDescriptor | Deprecated |
ImmutableHRegionInfo | Deprecated |
ImmutableHTableDescriptor | Deprecated |
Increment |
Used to perform Increment operations on a single row.
|
MasterCallable<V> |
A RetryingCallable for Master RPC operations.
|
MasterCoprocessorRpcChannelImpl |
The implementation of a master based coprocessor rpc channel.
|
MetaCache |
A cache implementation for region locations from meta.
|
MetricsConnection |
This class is for maintaining the various connection statistics and publishing them through
the metrics interfaces.
|
MetricsConnection.CallStats |
A container class for collecting details about the RPC call as it percolates.
|
MetricsConnection.CallTracker | |
MetricsConnection.RegionStats | |
MetricsConnection.RunnerStats | |
MultiAction |
Container for Actions (i.e.
|
MultiResponse |
A container for Result objects, grouped by regionName.
|
MultiResponse.RegionResult | |
MultiServerCallable |
Callable that handles the
multi method call going against a single
regionserver; i.e. |
Mutation | |
Mutation.CellWrapper | |
NoncedRegionServerCallable<T> |
Implementations make an rpc call against a RegionService via a protobuf Service.
|
NoOpRetryableCallerInterceptor |
Class that acts as a NoOpInterceptor.
|
NoOpRetryingInterceptorContext | |
Operation |
Superclass for any type that maps to a potentially application-level query.
|
OperationWithAttributes | |
PackagePrivateFieldAccessor |
A helper class used to access the package private field in o.a.h.h.client package.
|
PerClientRandomNonceGenerator |
NonceGenerator implementation that uses client ID hash + random int as nonce group, and random
numbers as nonces.
|
PreemptiveFastFailInterceptor |
The concrete
RetryingCallerInterceptor class that implements the preemptive fast fail
feature. |
Put |
Used to perform Put operations for a single row.
|
Query |
Base class for HBase read operations; e.g.
|
QuotaStatusCalls |
Client class to wrap RPCs to HBase servers for space quota status information.
|
RawAsyncHBaseAdmin |
The implementation of AsyncAdmin.
|
RawAsyncHBaseAdmin.AddColumnFamilyProcedureBiConsumer | |
RawAsyncHBaseAdmin.CreateNamespaceProcedureBiConsumer | |
RawAsyncHBaseAdmin.CreateTableProcedureBiConsumer | |
RawAsyncHBaseAdmin.DeleteColumnFamilyProcedureBiConsumer | |
RawAsyncHBaseAdmin.DeleteNamespaceProcedureBiConsumer | |
RawAsyncHBaseAdmin.DisableTableProcedureBiConsumer | |
RawAsyncHBaseAdmin.EnableTableProcedureBiConsumer | |
RawAsyncHBaseAdmin.MergeTableRegionProcedureBiConsumer | |
RawAsyncHBaseAdmin.ModifyColumnFamilyProcedureBiConsumer | |
RawAsyncHBaseAdmin.ModifyNamespaceProcedureBiConsumer | |
RawAsyncHBaseAdmin.ModifyTableProcedureBiConsumer | |
RawAsyncHBaseAdmin.NamespaceProcedureBiConsumer | |
RawAsyncHBaseAdmin.ProcedureBiConsumer | |
RawAsyncHBaseAdmin.SplitTableRegionProcedureBiConsumer | |
RawAsyncHBaseAdmin.TableProcedureBiConsumer | |
RawAsyncHBaseAdmin.TruncateTableProcedureBiConsumer | |
RawAsyncTableImpl |
The implementation of RawAsyncTable.
|
RegionAdminServiceCallable<T> |
Similar to RegionServerCallable but for the AdminService interface.
|
RegionCoprocessorRpcChannel |
Provides clients with an RPC connection to call Coprocessor Endpoint
Service s
against a given table region. |
RegionCoprocessorRpcChannelImpl |
The implementation of a region based coprocessor rpc channel.
|
RegionCoprocessorServiceExec |
Represents a coprocessor service method execution against a single region.
|
RegionInfoBuilder | |
RegionInfoBuilder.MutableRegionInfo |
An implementation of RegionInfo that adds mutable methods so can build a RegionInfo instance.
|
RegionInfoDisplay |
Utility used composing RegionInfo for 'display'; e.g.
|
RegionLoadStats |
POJO representing region server load
|
RegionReplicaUtil |
Utility methods which contain the logic for regions and replicas.
|
RegionServerCallable<T,S> |
Implementations make a RPC call against a RegionService via a protobuf Service.
|
RegionServerCoprocessorRpcChannelImpl |
The implementation of a region server based coprocessor rpc channel.
|
RequestControllerFactory |
A factory class that constructs an
RequestController . |
Result | |
ResultBoundedCompletionService<V> |
A completion service for the RpcRetryingCallerFactory.
|
ResultStatsUtil |
A
Result with some statistics about the server/region status |
RetriesExhaustedException.ThrowableWithExtraContext |
Datastructure that allows adding more info around Throwable incident.
|
RetryingCallerInterceptor |
This class is designed to fit into the RetryingCaller class which forms the
central piece of intelligence for the client side retries for most calls.
|
RetryingCallerInterceptorContext |
The context object used in the
RpcRetryingCaller to enable
RetryingCallerInterceptor to intercept calls. |
RetryingCallerInterceptorFactory |
Factory implementation to provide the
ConnectionImplementation with
the implementation of the RetryingCallerInterceptor that we would use
to intercept the RpcRetryingCaller during the course of their calls. |
RetryingTimeTracker |
Tracks the amount of time remaining for an operation.
|
ReversedClientScanner |
A reversed client scanner which support backward scanning
|
ReversedScannerCallable |
A reversed ScannerCallable which supports backward scanning.
|
RowMutations |
Performs multiple mutations atomically on a single row.
|
RpcRetryingCallable<V> |
A RetryingCallable for RPC connection operations.
|
RpcRetryingCallerFactory |
Factory to create an
RpcRetryingCaller |
RpcRetryingCallerImpl<T> |
Runs an rpc'ing
RetryingCallable . |
RpcRetryingCallerWithReadReplicas |
Caller that goes to replica if the primary region does no answer within a configurable
timeout.
|
Scan |
Used to perform Scan operations.
|
ScannerCallable |
Scanner operations such as create, next, etc.
|
ScannerCallableWithReplicas |
This class has the logic for handling scanners for regions with and without replicas.
|
SecureBulkLoadClient |
Client proxy for SecureBulkLoadProtocol
|
ServerStatisticTracker |
Tracks the statistics for multiple regions
|
ShortCircuitMasterConnection |
A short-circuit connection that can bypass the RPC layer (serialization, deserialization,
networking, etc..) when talking to a local master
|
SimpleRequestController |
Holds back the requests if they reach any thresholds.
|
SimpleRequestController.RequestHeapSizeChecker |
limit the heap size for each request.
|
SimpleRequestController.RequestRowsChecker |
limit the number of rows for each request.
|
SimpleRequestController.SubmittedSizeChecker |
limit the heapsize of total submitted data.
|
SimpleRequestController.TaskCountChecker |
limit the max number of tasks in an AsyncProcess.
|
SingleResponse |
Class for single action response
|
SingleResponse.Entry | |
SnapshotDescription |
The POJO equivalent of HBaseProtos.SnapshotDescription
|
SyncCoprocessorRpcChannel |
Base class which provides clients with an RPC connection to
call coprocessor endpoint
Service s. |
TableBuilderBase |
Base class for all table builders.
|
TableDescriptorBuilder | |
TableDescriptorBuilder.ModifyableTableDescriptor |
TODO: make this private after removing the HTableDescriptor
|
TableSnapshotScanner |
A Scanner which performs a scan over snapshot files.
|
TableState |
Represents table state.
|
UnmodifyableHRegionInfo | Deprecated
As of release 2.0.0, this will be removed in HBase 3.0.0.
|
VersionInfoUtil |
Class to help with parsing the version info.
|
ZKAsyncRegistry |
Fetch the registry data from zookeeper.
|
Enum | Description |
---|---|
AbstractResponse.ResponseType | |
AsyncProcessTask.SubmittedRows |
The number of processed rows.
|
AsyncRequestFutureImpl.Retry | |
AsyncScanSingleRegionRpcRetryingCaller.ScanControllerState | |
AsyncScanSingleRegionRpcRetryingCaller.ScanResumerState | |
CompactionState |
POJO representing the compaction state
|
CompactType |
Currently, there are only two compact types:
NORMAL means do store files compaction;
MOB means do mob files compaction. |
Consistency |
Consistency defines the expected consistency level for an operation.
|
Durability |
Enum describing the durability guarantees for tables and
Mutation s
Note that the items must be sorted in order of increasing durability |
IsolationLevel |
Specify Isolation levels in Scan operations.
|
MasterSwitchType |
Represents the master switch type
|
MobCompactPartitionPolicy |
Enum describing the mob compact partition policy types.
|
RegionLocateType |
Indicate which row you want to locate.
|
RequestController.ReturnCode | |
Scan.ReadType | |
ScannerCallable.MoreResults | |
SnapshotType |
POJO representing the snapshot type
|
TableState.State |
Exception | Description |
---|---|
DoNotRetryRegionException |
Similar to RegionException, but disables retries.
|
NoServerForRegionException |
Thrown when no region server can be found for a region
|
RegionOfflineException |
Thrown when a table can not be located
|
RetriesExhaustedException |
Exception thrown by HTable methods when an attempt to do something (like
commit changes) fails after a bunch of retries.
|
RetriesExhaustedWithDetailsException |
This subclass of
RetriesExhaustedException
is thrown when we have more information about which rows were causing which
exceptions on what servers. |
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).
|
WrongRowIOException |
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
.
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.
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.
Copyright © 2007–2019 The Apache Software Foundation. All rights reserved.