HBASE-24144 Update docs from master

Bring back documentation from master branch (2ecbaf130e), using

```
$ git checkout master -- src/main/asciidoc/
$ git checkout master -- src/site/asciidoc/
```

And then:
 * remove changes from HBASE-23890
   This reverts commit 420e38083f.
 * delete docs re: sync_replication
 * delete docs re: backup
This commit is contained in:
Nick Dimiduk 2020-06-15 12:17:00 -07:00 committed by Nick Dimiduk
parent 035ea516e9
commit c592b50b7b
38 changed files with 3626 additions and 1054 deletions

View File

@ -106,11 +106,11 @@ In the version 2 every block in the data section contains the following fields:
.. BLOOM_CHUNK Bloom filter chunks
.. META meta blocks (not used for Bloom filters in version 2 anymore)
.. INTERMEDIATE_INDEX intermediate-level index blocks in a multi-level blockindex
.. ROOT_INDEX root>level index blocks in a multi>level block index
.. FILE_INFO the ``file info'' block, a small key>value map of metadata
.. BLOOM_META a Bloom filter metadata block in the load>on>open section
.. TRAILER a fixed>size file trailer.
As opposed to the above, this is not an HFile v2 block but a fixed>size (for each HFile version) data structure
.. ROOT_INDEX root-level index blocks in a multi-level block index
.. FILE_INFO the ''file info'' block, a small key-value map of metadata
.. BLOOM_META a Bloom filter metadata block in the load-on-open section
.. TRAILER a fixed-size file trailer.
As opposed to the above, this is not an HFile v2 block but a fixed-size (for each HFile version) data structure
.. INDEX_V1 this block type is only used for legacy HFile v1 block
. Compressed size of the block's data, not including the header (int).
+
@ -127,7 +127,7 @@ The above format of blocks is used in the following HFile sections:
Scanned block section::
The section is named so because it contains all data blocks that need to be read when an HFile is scanned sequentially.
Also contains leaf block index and Bloom chunk blocks.
Also contains Leaf index blocks and Bloom chunk blocks.
Non-scanned block section::
This section still contains unified-format v2 blocks but it does not have to be read when doing a sequential scan.
This section contains "meta" blocks and intermediate-level index blocks.
@ -140,10 +140,10 @@ There are three types of block indexes in HFile version 2, stored in two differe
. Data index -- version 2 multi-level block index, consisting of:
.. Version 2 root index, stored in the data block index section of the file
.. Optionally, version 2 intermediate levels, stored in the non%root format in the data index section of the file. Intermediate levels can only be present if leaf level blocks are present
.. Optionally, version 2 leaf levels, stored in the non%root format inline with data blocks
.. Optionally, version 2 intermediate levels, stored in the non-root format in the data index section of the file. Intermediate levels can only be present if leaf level blocks are present
.. Optionally, version 2 leaf levels, stored in the non-root format inline with data blocks
. Meta index -- version 2 root index format only, stored in the meta index section of the file
. Bloom index -- version 2 root index format only, stored in the ``load-on-open'' section as part of Bloom filter metadata.
. Bloom index -- version 2 root index format only, stored in the ''load-on-open'' section as part of Bloom filter metadata.
==== Root block index format in version 2
@ -156,7 +156,7 @@ A version 2 root index block is a sequence of entries of the following format, s
. Offset (long)
+
This offset may point to a data block or to a deeper>level index block.
This offset may point to a data block or to a deeper-level index block.
. On-disk size (int)
. Key (a serialized byte array stored using Bytes.writeByteArray)
@ -172,7 +172,7 @@ For the data index and the meta index the number of entries is stored in the tra
For a multi-level block index we also store the following fields in the root index block in the load-on-open section of the HFile, in addition to the data structure described above:
. Middle leaf index block offset
. Middle leaf block on-disk size (meaning the leaf index block containing the reference to the ``middle'' data block of the file)
. Middle leaf block on-disk size (meaning the leaf index block containing the reference to the ''middle'' data block of the file)
. The index of the mid-key (defined below) in the middle leaf-level block.
@ -200,9 +200,9 @@ Every non-root index block is structured as follows.
. Entries.
Each entry contains:
+
. Offset of the block referenced by this entry in the file (long)
. On>disk size of the referenced block (int)
. Key.
.. Offset of the block referenced by this entry in the file (long)
.. On-disk size of the referenced block (int)
.. Key.
The length can be calculated from entryOffsets.
@ -214,7 +214,7 @@ In contrast with version 1, in a version 2 HFile Bloom filter metadata is stored
+
. Bloom filter version = 3 (int). There used to be a DynamicByteBloomFilter class that had the Bloom filter version number 2
. The total byte size of all compound Bloom filter chunks (long)
. Number of hash functions (int
. Number of hash functions (int)
. Type of hash functions (int)
. The total key count inserted into the Bloom filter (long)
. The maximum total number of keys in the Bloom filter (long)
@ -246,7 +246,7 @@ This is because we need to know the comparator at the time of parsing the load-o
==== Fixed file trailer format differences between versions 1 and 2
The following table shows common and different fields between fixed file trailers in versions 1 and 2.
Note that the size of the trailer is different depending on the version, so it is ``fixed'' only within one version.
Note that the size of the trailer is different depending on the version, so it is ''fixed'' only within one version.
However, the version is always stored as the last four-byte integer in the file.
.Differences between HFile Versions 1 and 2

View File

@ -260,6 +260,93 @@ For region name, we only accept `byte[]` as the parameter type and it may be a f
Information on non-Java clients and custom protocols is covered in <<external_apis>>
[[client.masterregistry]]
=== Master Registry (new as of 2.3.0)
Client internally works with a _connection registry_ to fetch the metadata needed by connections.
This connection registry implementation is responsible for fetching the following metadata.
* Active master address
* Current meta region(s) locations
* Cluster ID (unique to this cluster)
This information is needed as a part of various client operations like connection set up, scans,
gets, etc. Traditionally, the connection registry implementation has been based on ZooKeeper as the
source of truth and clients fetched the metadata directly from the ZooKeeper quorum. HBase 2.3.0
introduces a new connection registry implementation based on direct communication with the Masters.
With this implementation, clients now fetch required metadata via master RPC end points instead of
maintaining connections to ZooKeeper. This change was done for the following reasons.
* Reduce load on ZooKeeper since that is critical for cluster operation.
* Holistic client timeout and retry configurations since the new registry brings all the client
operations under HBase rpc framework.
* Remove the ZooKeeper client dependency on HBase client library.
This means:
* At least a single active or stand by master is needed for cluster connection setup. Refer to
<<master.runtime>> for more details.
* Master can be in a critical path of read/write operations, especially if the client metadata cache
is empty or stale.
* There is higher connection load on the masters that before since the clients talk directly to
HMasters instead of ZooKeeper ensemble`
To reduce hot-spotting on a single master, all the masters (active & stand-by) expose the needed
service to fetch the connection metadata. This lets the client connect to any master (not just active).
Both ZooKeeper- and Master-based connection registry implementations are available in 2.3+. For
2.3 and earlier, the ZooKeeper-based implementation remains the default configuration.
The Master-based implementation becomes the default in 3.0.0.
Change the connection registry implementation by updating the value configured for
`hbase.client.registry.impl`. To explicitly enable the ZooKeeper-based registry, use
[source, xml]
<property>
<name>hbase.client.registry.impl</name>
<value>org.apache.hadoop.hbase.client.ZKConnectionRegistry</value>
</property>
To explicitly enable the Master-based registry, use
[source, xml]
<property>
<name>hbase.client.registry.impl</name>
<value>org.apache.hadoop.hbase.client.MasterRegistry</value>
</property>
==== MasterRegistry RPC hedging
MasterRegistry implements hedging of connection registry RPCs across active and stand-by masters.
This lets the client make the same request to multiple servers and which ever responds first is
returned back to the client immediately. This improves performance, especially when a subset of
servers are under load. The hedging fan out size is configurable, meaning the number of requests
that are hedged in a single attempt, using the configuration key
_hbase.client.master_registry.hedged.fanout_ in the client configuration. It defaults to 2. With
this default, the RPCs are tried in batches of 2. The hedging policy is still primitive and does not
adapt to any sort of live rpc performance metrics.
==== Additional Notes
* Clients hedge the requests in a randomized order to avoid hot-spotting a single master.
* Cluster internal connections (masters &lt;-&gt; regionservers) still use ZooKeeper based connection
registry.
* Cluster internal state is still tracked in Zookeeper, hence ZK availability requirements are same
as before.
* Inter cluster replication still uses ZooKeeper based connection registry to simplify configuration
management.
For more implementation details, please refer to the https://github.com/apache/hbase/tree/master/dev-support/design-docs[design doc] and
https://issues.apache.org/jira/browse/HBASE-18095[HBASE-18095].
'''
NOTE: (Advanced) In case of any issues with the master based registry, use the following
configuration to fallback to the ZooKeeper based connection registry implementation.
[source, xml]
<property>
<name>hbase.client.registry.impl</name>
<value>org.apache.hadoop.hbase.client.ZKConnectionRegistry</value>
</property>
[[client.filter]]
== Client Request Filters
@ -466,8 +553,8 @@ scan.setFilter(f);
scan.setBatch(10); // set this if there could be many columns returned
ResultScanner rs = t.getScanner(scan);
for (Result r = rs.next(); r != null; r = rs.next()) {
for (KeyValue kv : r.raw()) {
// each kv represents a column
for (Cell cell : result.listCells()) {
// each cell represents a column
}
}
rs.close();
@ -496,8 +583,8 @@ scan.setFilter(f);
scan.setBatch(10); // set this if there could be many columns returned
ResultScanner rs = t.getScanner(scan);
for (Result r = rs.next(); r != null; r = rs.next()) {
for (KeyValue kv : r.raw()) {
// each kv represents a column
for (Cell cell : result.listCells()) {
// each cell represents a column
}
}
rs.close();
@ -532,8 +619,8 @@ scan.setFilter(f);
scan.setBatch(10); // set this if there could be many columns returned
ResultScanner rs = t.getScanner(scan);
for (Result r = rs.next(); r != null; r = rs.next()) {
for (KeyValue kv : r.raw()) {
// each kv represents a column
for (Cell cell : result.listCells()) {
// each cell represents a column
}
}
rs.close();
@ -577,11 +664,24 @@ If the active Master loses its lease in ZooKeeper (or the Master shuts down), th
[[master.runtime]]
=== Runtime Impact
A common dist-list question involves what happens to an HBase cluster when the Master goes down.
A common dist-list question involves what happens to an HBase cluster when the Master goes down. This information has changed staring 3.0.0.
==== Up until releases 2.x.y
Because the HBase client talks directly to the RegionServers, the cluster can still function in a "steady state". Additionally, per <<arch.catalog>>, `hbase:meta` exists as an HBase table and is not resident in the Master.
However, the Master controls critical functions such as RegionServer failover and completing region splits.
So while the cluster can still run for a short time without the Master, the Master should be restarted as soon as possible.
==== Staring release 3.0.0
As mentioned in section <<client.masterregistry>>, the default connection registry for clients is now based on master rpc end points. Hence the requirements for
masters' uptime are even tighter starting this release.
- At least one active or stand by master is needed for a connection set up, unlike before when all the clients needed was a ZooKeeper ensemble.
- Master is now in critical path for read/write operations. For example, if the meta region bounces off to a different region server, clients
need master to fetch the new locations. Earlier this was done by fetching this information directly from ZooKeeper.
- Masters will now have higher connection load than before. So, the server side configuration might need adjustment depending on the load.
Overall, the master uptime requirements, when this feature is enabled, are even higher for the client operations to go through.
[[master.api]]
=== Interface
@ -610,6 +710,83 @@ See <<regions.arch.assignment>> for more information on region assignment.
Periodically checks and cleans up the `hbase:meta` table.
See <<arch.catalog.meta>> for more information on the meta table.
[[master.wal]]
=== MasterProcWAL
_MasterProcWAL is replaced in hbase-2.3.0 by an alternate Procedure Store implementation; see
<<in-master-procedure-store-region>>. This section pertains to hbase-2.0.0 through hbase-2.2.x_
HMaster records administrative operations and their running states, such as the handling of a crashed server,
table creation, and other DDLs, into a Procedure Store. The Procedure Store WALs are stored under the
MasterProcWALs directory. The Master WALs are not like RegionServer WALs. Keeping up the Master WAL allows
us run a state machine that is resilient across Master failures. For example, if a HMaster was in the
middle of creating a table encounters an issue and fails, the next active HMaster can take up where
the previous left off and carry the operation to completion. Since hbase-2.0.0, a
new AssignmentManager (A.K.A AMv2) was introduced and the HMaster handles region assignment
operations, server crash processing, balancing, etc., all via AMv2 persisting all state and
transitions into MasterProcWALs rather than up into ZooKeeper, as we do in hbase-1.x.
See <<amv2>> (and <<pv2>> for its basis) if you would like to learn more about the new
AssignmentManager.
[[master.wal.conf]]
==== Configurations for MasterProcWAL
Here are the list of configurations that effect MasterProcWAL operation.
You should not have to change your defaults.
[[hbase.procedure.store.wal.periodic.roll.msec]]
*`hbase.procedure.store.wal.periodic.roll.msec`*::
+
.Description
Frequency of generating a new WAL
+
.Default
`1h (3600000 in msec)`
[[hbase.procedure.store.wal.roll.threshold]]
*`hbase.procedure.store.wal.roll.threshold`*::
+
.Description
Threshold in size before the WAL rolls. Every time the WAL reaches this size or the above period, 1 hour, passes since last log roll, the HMaster will generate a new WAL.
+
.Default
`32MB (33554432 in byte)`
[[hbase.procedure.store.wal.warn.threshold]]
*`hbase.procedure.store.wal.warn.threshold`*::
+
.Description
If the number of WALs goes beyond this threshold, the following message should appear in the HMaster log with WARN level when rolling.
procedure WALs count=xx above the warning threshold 64. check running procedures to see if something is stuck.
+
.Default
`64`
[[hbase.procedure.store.wal.max.retries.before.roll]]
*`hbase.procedure.store.wal.max.retries.before.roll`*::
+
.Description
Max number of retry when syncing slots (records) to its underlying storage, such as HDFS. Every attempt, the following message should appear in the HMaster log.
unable to sync slots, retry=xx
+
.Default
`3`
[[hbase.procedure.store.wal.sync.failure.roll.max]]
*`hbase.procedure.store.wal.sync.failure.roll.max`*::
+
.Description
After the above 3 retrials, the log is rolled and the retry count is reset to 0, thereon a new set of retrial starts. This configuration controls the max number of attempts of log rolling upon sync failure. That is, HMaster is allowed to fail to sync 9 times in total. Once it exceeds, the following log should appear in the HMaster log.
Sync slots after log roll failed, abort.
+
.Default
`3`
[[regionserver.arch]]
== RegionServer
@ -779,7 +956,7 @@ Here are two use cases:
Setting block caching on such a table is a waste of memory and CPU cycles, more so that it will generate more garbage to pick up by the JVM.
For more information on monitoring GC, see <<trouble.log.gc>>.
* Mapping a table: In a typical MapReduce job that takes a table in input, every row will be read only once so there's no need to put them into the block cache.
The Scan object has the option of turning this off via the setCaching method (set it to false). You can still keep block caching turned on on this table if you need fast random read access.
The Scan object has the option of turning this off via the setCacheBlocks method (set it to false). You can still keep block caching turned on on this table if you need fast random read access.
An example would be counting the number of rows in a table that serves live traffic, caching every block of that table would create massive churn and would surely evict data that's currently in use.
[[data.blocks.in.fscache]]
@ -831,8 +1008,9 @@ benefit of NOT provoking GC.
From HBase 2.0.0 onwards, the notions of L1 and L2 have been deprecated. When BucketCache is turned on, the DATA blocks will always go to BucketCache and INDEX/BLOOM blocks go to on heap LRUBlockCache. `cacheDataInL1` support hase been removed.
====
The BucketCache Block Cache can be deployed _off-heap_, _file_ or _mmaped_ file mode.
[[bc.deloy.modes]]
====== BucketCache Deploy Modes
The BucketCache Block Cache can be deployed _offheap_, _file_ or _mmaped_ file mode.
You set which via the `hbase.bucketcache.ioengine` setting.
Setting it to `offheap` will have BucketCache make its allocations off-heap, and an ioengine setting of `file:PATH_TO_FILE` will direct BucketCache to use file caching (Useful in particular if you have some fast I/O attached to the box such as SSDs). From 2.0.0, it is possible to have more than one file backing the BucketCache. This is very useful specially when the Cache size requirement is high. For multiple backing files, configure ioengine as `files:PATH_TO_FILE1,PATH_TO_FILE2,PATH_TO_FILE3`. BucketCache can be configured to use an mmapped file also. Configure ioengine as `mmap:PATH_TO_FILE` for this.
@ -851,6 +1029,7 @@ See the link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfil
To check it enabled, look for the log line describing cache setup; it will detail how BucketCache has been deployed.
Also see the UI. It will detail the cache tiering and their configuration.
[[bc.example]]
====== BucketCache Example Configuration
This sample provides a configuration for a 4 GB off-heap BucketCache with a 1 GB on-heap cache.
@ -894,14 +1073,14 @@ In the above, we set the BucketCache to be 4G.
We configured the on-heap LruBlockCache have 20% (0.2) of the RegionServer's heap size (0.2 * 5G = 1G). In other words, you configure the L1 LruBlockCache as you would normally (as if there were no L2 cache present).
link:https://issues.apache.org/jira/browse/HBASE-10641[HBASE-10641] introduced the ability to configure multiple sizes for the buckets of the BucketCache, in HBase 0.98 and newer.
To configurable multiple bucket sizes, configure the new property `hfile.block.cache.sizes` (instead of `hfile.block.cache.size`) to a comma-separated list of block sizes, ordered from smallest to largest, with no spaces.
To configurable multiple bucket sizes, configure the new property `hbase.bucketcache.bucket.sizes` to a comma-separated list of block sizes, ordered from smallest to largest, with no spaces.
The goal is to optimize the bucket sizes based on your data access patterns.
The following example configures buckets of size 4096 and 8192.
[source,xml]
----
<property>
<name>hfile.block.cache.sizes</name>
<name>hbase.bucketcache.bucket.sizes</name>
<value>4096,8192</value>
</property>
----
@ -1145,127 +1324,40 @@ WAL log splitting and recovery can be resource intensive and take a long time, d
Distributed log processing is enabled by default since HBase 0.92.
The setting is controlled by the `hbase.master.distributed.log.splitting` property, which can be set to `true` or `false`, but defaults to `true`.
[[log.splitting.step.by.step]]
.Distributed Log Splitting, Step by Step
==== WAL splitting based on procedureV2
After HBASE-20610, we introduce a new way to do WAL splitting coordination by procedureV2 framework. This can simplify the process of WAL splitting and no need to connect zookeeper any more.
After configuring distributed log splitting, the HMaster controls the process.
The HMaster enrolls each RegionServer in the log splitting process, and the actual work of splitting the logs is done by the RegionServers.
The general process for log splitting, as described in <<log.splitting.step.by.step>> still applies here.
[[background]]
.Background
Currently, splitting WAL processes are coordinated by zookeeper. Each region server are trying to grab tasks from zookeeper. And the burden becomes heavier when the number of region server increase.
. If distributed log processing is enabled, the HMaster creates a _split log manager_ instance when the cluster is started.
.. The split log manager manages all log files which need to be scanned and split.
.. The split log manager places all the logs into the ZooKeeper splitWAL node (_/hbase/splitWAL_) as tasks.
.. You can view the contents of the splitWAL by issuing the following `zkCli` command. Example output is shown.
+
[source,bash]
----
ls /hbase/splitWAL
[hdfs%3A%2F%2Fhost2.sample.com%3A56020%2Fhbase%2FWALs%2Fhost8.sample.com%2C57020%2C1340474893275-splitting%2Fhost8.sample.com%253A57020.1340474893900,
hdfs%3A%2F%2Fhost2.sample.com%3A56020%2Fhbase%2FWALs%2Fhost3.sample.com%2C57020%2C1340474893299-splitting%2Fhost3.sample.com%253A57020.1340474893931,
hdfs%3A%2F%2Fhost2.sample.com%3A56020%2Fhbase%2FWALs%2Fhost4.sample.com%2C57020%2C1340474893287-splitting%2Fhost4.sample.com%253A57020.1340474893946]
----
+
The output contains some non-ASCII characters.
When decoded, it looks much more simple:
+
----
[hdfs://host2.sample.com:56020/hbase/WALs
/host8.sample.com,57020,1340474893275-splitting
/host8.sample.com%3A57020.1340474893900,
hdfs://host2.sample.com:56020/hbase/WALs
/host3.sample.com,57020,1340474893299-splitting
/host3.sample.com%3A57020.1340474893931,
hdfs://host2.sample.com:56020/hbase/WALs
/host4.sample.com,57020,1340474893287-splitting
/host4.sample.com%3A57020.1340474893946]
----
+
The listing represents WAL file names to be scanned and split, which is a list of log splitting tasks.
[[implementation.on.master.side]]
.Implementation on Master side
During ServerCrashProcedure, SplitWALManager will create one SplitWALProcedure for each WAL file which should be split. Then each SplitWALProcedure will spawn a SplitWalRemoteProcedure to send the request to region server.
SplitWALProcedure is a StateMachineProcedure and here is the state transfer diagram.
. The split log manager monitors the log-splitting tasks and workers.
+
The split log manager is responsible for the following ongoing tasks:
+
* Once the split log manager publishes all the tasks to the splitWAL znode, it monitors these task nodes and waits for them to be processed.
* Checks to see if there are any dead split log workers queued up.
If it finds tasks claimed by unresponsive workers, it will resubmit those tasks.
If the resubmit fails due to some ZooKeeper exception, the dead worker is queued up again for retry.
* Checks to see if there are any unassigned tasks.
If it finds any, it create an ephemeral rescan node so that each split log worker is notified to re-scan unassigned tasks via the `nodeChildrenChanged` ZooKeeper event.
* Checks for tasks which are assigned but expired.
If any are found, they are moved back to `TASK_UNASSIGNED` state again so that they can be retried.
It is possible that these tasks are assigned to slow workers, or they may already be finished.
This is not a problem, because log splitting tasks have the property of idempotence.
In other words, the same log splitting task can be processed many times without causing any problem.
* The split log manager watches the HBase split log znodes constantly.
If any split log task node data is changed, the split log manager retrieves the node data.
The node data contains the current state of the task.
You can use the `zkCli` `get` command to retrieve the current state of a task.
In the example output below, the first line of the output shows that the task is currently unassigned.
+
----
get /hbase/splitWAL/hdfs%3A%2F%2Fhost2.sample.com%3A56020%2Fhbase%2FWALs%2Fhost6.sample.com%2C57020%2C1340474893287-splitting%2Fhost6.sample.com%253A57020.1340474893945
.WAL_splitting_coordination
image::WAL_splitting.png[]
unassigned host2.sample.com:57000
cZxid = 0×7115
ctime = Sat Jun 23 11:13:40 PDT 2012
...
----
+
Based on the state of the task whose data is changed, the split log manager does one of the following:
+
* Resubmit the task if it is unassigned
* Heartbeat the task if it is assigned
* Resubmit or fail the task if it is resigned (see <<distributed.log.replay.failure.reasons>>)
* Resubmit or fail the task if it is completed with errors (see <<distributed.log.replay.failure.reasons>>)
* Resubmit or fail the task if it could not complete due to errors (see <<distributed.log.replay.failure.reasons>>)
* Delete the task if it is successfully completed or failed
+
[[distributed.log.replay.failure.reasons]]
[NOTE]
.Reasons a Task Will Fail
====
* The task has been deleted.
* The node no longer exists.
* The log status manager failed to move the state of the task to `TASK_UNASSIGNED`.
* The number of resubmits is over the resubmit threshold.
====
[[implementation.on.region.server.side]]
.Implementation on Region Server side
Region Server will receive a SplitWALCallable and execute it, which is much more straightforward than before. It will return null if success and return exception if there is any error.
. Each RegionServer's split log worker performs the log-splitting tasks.
+
Each RegionServer runs a daemon thread called the _split log worker_, which does the work to split the logs.
The daemon thread starts when the RegionServer starts, and registers itself to watch HBase znodes.
If any splitWAL znode children change, it notifies a sleeping worker thread to wake up and grab more tasks.
If a worker's current task's node data is changed,
the worker checks to see if the task has been taken by another worker.
If so, the worker thread stops work on the current task.
+
The worker monitors the splitWAL znode constantly.
When a new task appears, the split log worker retrieves the task paths and checks each one until it finds an unclaimed task, which it attempts to claim.
If the claim was successful, it attempts to perform the task and updates the task's `state` property based on the splitting outcome.
At this point, the split log worker scans for another unclaimed task.
+
.How the Split Log Worker Approaches a Task
* It queries the task state and only takes action if the task is in `TASK_UNASSIGNED `state.
* If the task is in `TASK_UNASSIGNED` state, the worker attempts to set the state to `TASK_OWNED` by itself.
If it fails to set the state, another worker will try to grab it.
The split log manager will also ask all workers to rescan later if the task remains unassigned.
* If the worker succeeds in taking ownership of the task, it tries to get the task state again to make sure it really gets it asynchronously.
In the meantime, it starts a split task executor to do the actual work:
** Get the HBase root folder, create a temp folder under the root, and split the log file to the temp folder.
** If the split was successful, the task executor sets the task to state `TASK_DONE`.
** If the worker catches an unexpected IOException, the task is set to state `TASK_ERR`.
** If the worker is shutting down, set the task to state `TASK_RESIGNED`.
** If the task is taken by another worker, just log it.
[[preformance]]
.Performance
According to tests on a cluster which has 5 regionserver and 1 master.
procedureV2 coordinated WAL splitting has a better performance than ZK coordinated WAL splitting no master when restarting the whole cluster or one region server crashing.
[[enable.this.feature]]
.Enable this feature
To enable this feature, first we should ensure our package of HBase already contains these code. If not, please upgrade the package of HBase cluster without any configuration change first.
Then change configuration 'hbase.split.wal.zk.coordinated' to false. Rolling upgrade the master with new configuration. Now WAL splitting are handled by our new implementation.
But region server are still trying to grab tasks from zookeeper, we can rolling upgrade the region servers with the new configuration to stop that.
. The split log manager monitors for uncompleted tasks.
+
The split log manager returns when all tasks are completed successfully.
If all tasks are completed with some failures, the split log manager throws an exception so that the log splitting can be retried.
Due to an asynchronous implementation, in very rare cases, the split log manager loses track of some completed tasks.
For that reason, it periodically checks for remaining uncompleted task in its task map or ZooKeeper.
If none are found, it throws an exception so that the log splitting can be retried right away instead of hanging there waiting for something that won't happen.
* steps as follows:
** Upgrade whole cluster to get the new Implementation.
** Upgrade Master with new configuration 'hbase.split.wal.zk.coordinated'=false.
** Upgrade region server to stop grab tasks from zookeeper.
[[wal.compression]]
==== WAL Compression ====
@ -1295,6 +1387,26 @@ It is possible to set _durability_ on each Mutation or on a Table basis. Options
Do not confuse the _ASYNC_WAL_ option on a Mutation or Table with the _AsyncFSWAL_ writer; they are distinct
options unfortunately closely named
[[arch.custom.wal.dir]]
==== Custom WAL Directory
HBASE-17437 added support for specifying a WAL directory outside the HBase root directory or even in a different FileSystem since 1.3.3/2.0+. Some FileSystems (such as Amazon S3) dont support append or consistent writes, in such scenario WAL directory needs to be configured in a different FileSystem to avoid loss of writes.
Following configurations are added to accomplish this:
. `hbase.wal.dir`
+
This defines where the root WAL directory is located, could be on a different FileSystem than the root directory. WAL directory can not be set to a subdirectory of the root directory. The default value of this is the root directory if unset.
. `hbase.rootdir.perms`
+
Configures FileSystem permissions to set on the root directory. This is '700' by default.
. `hbase.wal.dir.perms`
+
Configures FileSystem permissions to set on the WAL directory FileSystem. This is '700' by default.
NOTE: While migrating to custom WAL dir (outside the HBase root directory or a different FileSystem) existing WAL files must be copied manually to new WAL dir, otherwise it may lead to data loss/inconsistency as HMaster has no information about previous WAL directory.
[[wal.disable]]
==== Disabling the WAL
@ -1681,6 +1793,9 @@ For example, to view the content of the file _hdfs://10.81.47.41:8020/hbase/defa
If you leave off the option -v to see just a summary on the HFile.
See usage for other things to do with the `hfile` tool.
NOTE: In the output of this tool, you might see 'seqid=0' for certain keys in places such as 'Mid-key'/'firstKey'/'lastKey'. These are
'KeyOnlyKeyValue' type instances - meaning their seqid is irrelevant & we just need the keys of these Key-Value instances.
[[store.file.dir]]
===== StoreFile Directory Structure on HDFS
@ -1794,8 +1909,8 @@ Instead, the expired data is filtered out and is not written back to the compact
[[compaction.and.versions]]
.Compaction and Versions
When you create a Column Family, you can specify the maximum number of versions to keep, by specifying `HColumnDescriptor.setMaxVersions(int versions)`.
The default value is `3`.
When you create a Column Family, you can specify the maximum number of versions to keep, by specifying `ColumnFamilyDescriptorBuilder.setMaxVersions(int versions)`.
The default value is `1`.
If more versions than the specified maximum exist, the excess versions are filtered out and not written back to the compacted StoreFile.
.Major Compactions Can Impact Query Results
@ -2436,14 +2551,8 @@ To control these for stripe compactions, use `hbase.store.stripe.compaction.minF
HBase includes several methods of loading data into tables.
The most straightforward method is to either use the `TableOutputFormat` class from a MapReduce job, or use the normal client APIs; however, these are not always the most efficient methods.
The bulk load feature uses a MapReduce job to output table data in HBase's internal data format, and then directly loads the generated StoreFiles into a running cluster.
Using bulk load will use less CPU and network resources than simply using the HBase API.
[[arch.bulk.load.limitations]]
=== Bulk Load Limitations
As bulk loading bypasses the write path, the WAL doesn't get written to as part of the process.
Replication works by reading the WAL files so it won't see the bulk loaded data and the same goes for the edits that use `Put.setDurability(SKIP_WAL)`. One way to handle that is to ship the raw files or the HFiles to the other cluster and do the other processing there.
The bulk load feature uses a MapReduce job to output table data in HBase's internal data format, and then directly load the generated StoreFiles into a running cluster.
Using bulk load will use less CPU and network resources than loading via the HBase API.
[[arch.bulk.load.arch]]
=== Bulk Load Architecture
@ -2454,7 +2563,7 @@ The HBase bulk load process consists of two main steps.
==== Preparing data via a MapReduce job
The first step of a bulk load is to generate HBase data files (StoreFiles) from a MapReduce job using `HFileOutputFormat2`.
This output format writes out data in HBase's internal storage format so that they can be later loaded very efficiently into the cluster.
This output format writes out data in HBase's internal storage format so that they can be later loaded efficiently into the cluster.
In order to function efficiently, `HFileOutputFormat2` must be configured such that each output HFile fits within a single region.
In order to do this, jobs whose output will be bulk loaded into HBase use Hadoop's `TotalOrderPartitioner` class to partition the map output into disjoint ranges of the key space, corresponding to the key ranges of the regions in the table.
@ -2471,22 +2580,20 @@ It then contacts the appropriate RegionServer which adopts the HFile, moving it
If the region boundaries have changed during the course of bulk load preparation, or between the preparation and completion steps, the `completebulkload` utility will automatically split the data files into pieces corresponding to the new boundaries.
This process is not optimally efficient, so users should take care to minimize the delay between preparing a bulk load and importing it into the cluster, especially if other clients are simultaneously loading data through other means.
[[arch.bulk.load.complete.help]]
[source,bash]
----
$ hadoop jar hbase-server-VERSION.jar completebulkload [-c /path/to/hbase/config/hbase-site.xml] /user/todd/myoutput mytable
$ hadoop jar hbase-mapreduce-VERSION.jar completebulkload [-c /path/to/hbase/config/hbase-site.xml] /user/todd/myoutput mytable
----
The `-c config-file` option can be used to specify a file containing the appropriate hbase parameters (e.g., hbase-site.xml) if not supplied already on the CLASSPATH (In addition, the CLASSPATH must contain the directory that has the zookeeper configuration file if zookeeper is NOT managed by HBase).
NOTE: If the target table does not already exist in HBase, this tool will create the table automatically.
[[arch.bulk.load.also]]
=== See Also
For more information about the referenced utilities, see <<importtsv>> and <<completebulkload>>.
See link:http://blog.cloudera.com/blog/2013/09/how-to-use-hbase-bulk-loading-and-why/[How-to: Use HBase Bulk Loading, and Why] for a recent blog on current state of bulk loading.
See link:http://blog.cloudera.com/blog/2013/09/how-to-use-hbase-bulk-loading-and-why/[How-to: Use HBase Bulk Loading, and Why] for an old blog post on loading.
[[arch.bulk.load.adv]]
=== Advanced Usage
@ -2497,6 +2604,79 @@ To get started doing so, dig into `ImportTsv.java` and check the JavaDoc for HFi
The import step of the bulk load can also be done programmatically.
See the `LoadIncrementalHFiles` class for more information.
[[arch.bulk.load.complete.strays]]
==== 'Adopting' Stray Data
Should an HBase cluster lose account of regions or files during an outage or error, you can use
the `completebulkload` tool to add back the dropped data. HBase operator tooling such as
link:https://github.com/apache/hbase-operator-tools/tree/master/hbase-hbck2[HBCK2] or
the reporting added to the Master's UI under the `HBCK Report` (Since HBase 2.0.6/2.1.6/2.2.1)
can identify such 'orphan' directories.
Before you begin the 'adoption', ensure the `hbase:meta` table is in a healthy state.
Run the `CatalogJanitor` by executing the `catalogjanitor_run` command on the HBase shell.
When finished, check the `HBCK Report` page on the Master UI. Work on fixing any
inconsistencies, holes, or overlaps found before proceeding. The `hbase:meta` table
is the authority on where all data is to be found and must be consistent for
the `completebulkload` tool to work properly.
The `completebulkload` tool takes a directory and a `tablename`.
The directory has subdirectories named for column families of the targeted `tablename`.
In these subdirectories are `hfiles` to load. Given this structure, you can pass
errant region directories (and the table name to which the region directory belongs)
and the tool will bring the data files back into the fold by moving them under the
approprate serving directory. If stray files, then you will need to mock up this
structure before invoking the `completebulkload` tool; you may have to look at the
file content using the <<hfile.tool>> to see what the column family to use is.
When the tool completes its run, you will notice that the
source errant directory has had its storefiles moved/removed. It is now desiccated
since its data has been drained, and the pointed-to directory can be safely
removed. It may still have `.regioninfo` files and other
subdirectories but they are of no relevance now (There may be content still
under the _recovered_edits_ directory; a TODO is tooling to replay the
content of _recovered_edits_ if needed; see
link:https://issues.apache.org/jira/browse/HBASE-22976[Add RecoveredEditsPlayer]).
If you pass `completebulkload` a directory without store files, it will run and
note the directory is storefile-free. Just remove such 'empty' directories.
For example, presuming a directory at the top level in HDFS named
`eb3352fb5c9c9a05feeb2caba101e1cc` has data we need to re-add to the
HBase `TestTable`:
[source,bash]
----
$ ${HBASE_HOME}/bin/hbase --config ~/hbase-conf completebulkload hdfs://server.example.org:9000/eb3352fb5c9c9a05feeb2caba101e1cc TestTable
----
After it successfully completes, any files that were in `eb3352fb5c9c9a05feeb2caba101e1cc` have been moved
under hbase and the `eb3352fb5c9c9a05feeb2caba101e1cc` directory can be deleted (Check content
before and after by running `ls -r` on the HDFS directory).
[[arch.bulk.load.replication]]
=== Bulk Loading Replication
HBASE-13153 adds replication support for bulk loaded HFiles, available since HBase 1.3/2.0. This feature is enabled by setting `hbase.replication.bulkload.enabled` to `true` (default is `false`).
You also need to copy the source cluster configuration files to the destination cluster.
Additional configurations are required too:
. `hbase.replication.source.fs.conf.provider`
+
This defines the class which loads the source cluster file system client configuration in the destination cluster. This should be configured for all the RS in the destination cluster. Default is `org.apache.hadoop.hbase.replication.regionserver.DefaultSourceFSConfigurationProvider`.
+
. `hbase.replication.conf.dir`
+
This represents the base directory where the file system client configurations of the source cluster are copied to the destination cluster. This should be configured for all the RS in the destination cluster. Default is `$HBASE_CONF_DIR`.
+
. `hbase.replication.cluster.id`
+
This configuration is required in the cluster where replication for bulk loaded data is enabled. A source cluster is uniquely identified by the destination cluster using this id. This should be configured for all the RS in the source cluster configuration file for all the RS.
+
For example: If source cluster FS client configurations are copied to the destination cluster under directory `/home/user/dc1/`, then `hbase.replication.cluster.id` should be configured as `dc1` and `hbase.replication.conf.dir` as `/home/user`.
NOTE: `DefaultSourceFSConfigurationProvider` supports only `xml` type files. It loads source cluster FS client configuration only once, so if source cluster FS client configuration files are updated, every peer(s) cluster RS must be restarted to reload the configuration.
[[arch.hdfs]]
== HDFS

View File

@ -82,20 +82,17 @@ NOTE: End-of-life releases are not included in this list.
| Release
| Release Manager
| 1.2
| Sean Busbey
| 1.3
| Mikhail Antonov
| 1.4
| Andrew Purtell
| 2.0
| Michael Stack
| 2.2
| Guanghao Zhang
| 2.1
| Duo Zhang
| 2.3
| Nick Dimiduk
|===

View File

@ -336,9 +336,7 @@ If you are changing codecs, be sure the old codec is still available until all t
.Enabling Compression on a ColumnFamily of an Existing Table using HBaseShell
----
hbase> disable 'test'
hbase> alter 'test', {NAME => 'cf', COMPRESSION => 'GZ'}
hbase> enable 'test'
----
.Creating a New Table with Compression On a ColumnFamily
@ -436,15 +434,12 @@ Following is an example using HBase Shell:
.Enable Data Block Encoding On a Table
----
hbase> disable 'test'
hbase> alter 'test', { NAME => 'cf', DATA_BLOCK_ENCODING => 'FAST_DIFF' }
Updating all regions with the new schema...
0/1 regions updated.
1/1 regions updated.
Done.
0 row(s) in 2.2820 seconds
hbase> enable 'test'
0 row(s) in 0.1580 seconds
----
.Verifying a ColumnFamily's Data Block Encoding

File diff suppressed because it is too large Load Diff

View File

@ -139,7 +139,7 @@ Referential Integrity::
Secondary Indexes::
You can use a coprocessor to maintain secondary indexes. For more information, see
link:https://wiki.apache.org/hadoop/Hbase/SecondaryIndexing[SecondaryIndexing].
link:https://cwiki.apache.org/confluence/display/HADOOP2/Hbase+SecondaryIndexing[SecondaryIndexing].
==== Types of Observer Coprocessor
@ -178,11 +178,28 @@ average or summation for an entire table which spans hundreds of regions.
In contrast to observer coprocessors, where your code is run transparently, endpoint
coprocessors must be explicitly invoked using the
link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Table.html#coprocessorService-java.lang.Class-byte:A-byte:A-org.apache.hadoop.hbase.client.coprocessor.Batch.Call-[CoprocessorService()]
link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/AsyncTable.html#coprocessorService-java.util.function.Function-org.apache.hadoop.hbase.client.ServiceCaller-byte:A-[CoprocessorService()]
method available in
link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Table.html[Table]
or
link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTable.html[HTable].
link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/AsyncTable.html[AsyncTable].
[WARNING]
.On using coprocessorService method with sync client
====
The coprocessorService method in link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Table.html[Table]
has been deprecated.
In link:https://issues.apache.org/jira/browse/HBASE-21512[HBASE-21512]
we reimplement the sync client based on the async client. The coprocessorService
method defined in `Table` interface directly references a method from protobuf's
`BlockingInterface`, which means we need to use a separate thread pool to execute
the method so we avoid blocking the async client(We want to avoid blocking calls in
our async implementation).
Since coprocessor is an advanced feature, we believe it is OK for coprocessor users to
instead switch over to use `AsyncTable`. There is a lightweight
link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Connection.html#toAsyncConnection--[toAsyncConnection]
method to get an `AsyncConnection` from `Connection` if needed.
====
Starting with HBase 0.96, endpoint coprocessors are implemented using Google Protocol
Buffers (protobuf). For more details on protobuf, see Google's
@ -193,6 +210,12 @@ link:https://issues.apache.org/jira/browse/HBASE-5448[HBASE-5448]). To upgrade y
HBase cluster from 0.94 or earlier to 0.96 or later, you need to reimplement your
coprocessor.
In HBase 2.x, we made use of a shaded version of protobuf 3.x, but kept the
protobuf for coprocessors on 2.5.0. In HBase 3.0.0, we removed all dependencies on
non-shaded protobuf so you need to reimplement your coprocessor to make use of the
shaded protobuf version provided in hbase-thirdparty. Please see
the <<protobuf,protobuf>> section for more details.
Coprocessor Endpoints should make no use of HBase internals and
only avail of public APIs; ideally a CPEP should depend on Interfaces
and data structures only. This is not always possible but beware
@ -310,13 +333,6 @@ dependencies.
[[load_coprocessor_in_shell]]
==== Using HBase Shell
. Disable the table using HBase Shell:
+
[source]
----
hbase> disable 'users'
----
. Load the Coprocessor, using a command like the following:
+
[source]
@ -346,12 +362,6 @@ observers registered at the same hook using priorities. This field can be left b
case the framework will assign a default priority value.
* Arguments (Optional): This field is passed to the Coprocessor implementation. This is optional.
. Enable the table.
+
----
hbase(main):003:0> enable 'users'
----
. Verify that the coprocessor loaded:
+
----
@ -372,7 +382,6 @@ String path = "hdfs://<namenode>:<port>/user/<hadoop-user>/coprocessor.jar";
Configuration conf = HBaseConfiguration.create();
Connection connection = ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
admin.disableTable(tableName);
HTableDescriptor hTableDescriptor = new HTableDescriptor(tableName);
HColumnDescriptor columnFamily1 = new HColumnDescriptor("personalDet");
columnFamily1.setMaxVersions(3);
@ -384,7 +393,6 @@ hTableDescriptor.setValue("COPROCESSOR$1", path + "|"
+ RegionObserverExample.class.getCanonicalName() + "|"
+ Coprocessor.PRIORITY_USER);
admin.modifyTable(tableName, hTableDescriptor);
admin.enableTable(tableName);
----
==== Using the Java API (HBase 0.96+ only)
@ -399,7 +407,6 @@ Path path = new Path("hdfs://<namenode>:<port>/user/<hadoop-user>/coprocessor.ja
Configuration conf = HBaseConfiguration.create();
Connection connection = ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
admin.disableTable(tableName);
HTableDescriptor hTableDescriptor = new HTableDescriptor(tableName);
HColumnDescriptor columnFamily1 = new HColumnDescriptor("personalDet");
columnFamily1.setMaxVersions(3);
@ -410,7 +417,6 @@ hTableDescriptor.addFamily(columnFamily2);
hTableDescriptor.addCoprocessor(RegionObserverExample.class.getCanonicalName(), path,
Coprocessor.PRIORITY_USER, null);
admin.modifyTable(tableName, hTableDescriptor);
admin.enableTable(tableName);
----
WARNING: There is no guarantee that the framework will load a given Coprocessor successfully.
@ -422,13 +428,6 @@ verifies whether the given class is actually contained in the jar file.
==== Using HBase Shell
. Disable the table.
+
[source]
----
hbase> disable 'users'
----
. Alter the table to remove the coprocessor.
+
[source]
@ -436,13 +435,6 @@ hbase> disable 'users'
hbase> alter 'users', METHOD => 'table_att_unset', NAME => 'coprocessor$1'
----
. Enable the table.
+
[source]
----
hbase> enable 'users'
----
==== Using the Java API
Reload the table definition without setting the value of the coprocessor either by
@ -456,7 +448,6 @@ String path = "hdfs://<namenode>:<port>/user/<hadoop-user>/coprocessor.jar";
Configuration conf = HBaseConfiguration.create();
Connection connection = ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
admin.disableTable(tableName);
HTableDescriptor hTableDescriptor = new HTableDescriptor(tableName);
HColumnDescriptor columnFamily1 = new HColumnDescriptor("personalDet");
columnFamily1.setMaxVersions(3);
@ -465,7 +456,6 @@ HColumnDescriptor columnFamily2 = new HColumnDescriptor("salaryDet");
columnFamily2.setMaxVersions(3);
hTableDescriptor.addFamily(columnFamily2);
admin.modifyTable(tableName, hTableDescriptor);
admin.enableTable(tableName);
----
In HBase 0.96 and newer, you can instead use the `removeCoprocessor()` method of the
@ -499,6 +489,7 @@ The following Observer coprocessor prevents the details of the user `admin` from
returned in a `Get` or `Scan` of the `users` table.
. Write a class that implements the
link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionCoprocessor.html[RegionCoprocessor],
link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html[RegionObserver]
class.
@ -516,16 +507,20 @@ empty result. Otherwise, process the request as normal.
Following are the implementation of the above steps:
[source,java]
----
public class RegionObserverExample implements RegionObserver {
public class RegionObserverExample implements RegionCoprocessor, RegionObserver {
private static final byte[] ADMIN = Bytes.toBytes("admin");
private static final byte[] COLUMN_FAMILY = Bytes.toBytes("details");
private static final byte[] COLUMN = Bytes.toBytes("Admin_det");
private static final byte[] VALUE = Bytes.toBytes("You can't see Admin details");
@Override
public Optional<RegionObserver> getRegionObserver() {
return Optional.of(this);
}
@Override
public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e, final Get get, final List<Cell> results)
throws IOException {

View File

@ -425,7 +425,7 @@ Get get = new Get(Bytes.toBytes("row1"));
get.setMaxVersions(3); // will return last 3 versions of row
Result r = table.get(get);
byte[] b = r.getValue(CF, ATTR); // returns current version of value
List<KeyValue> kv = r.getColumn(CF, ATTR); // returns all versions of this column
List<Cell> cells = r.getColumnCells(CF, ATTR); // returns all versions of this column
----
==== Put
@ -471,6 +471,26 @@ Caution: the version timestamp is used internally by HBase for things like time-
It's usually best to avoid setting this timestamp yourself.
Prefer using a separate timestamp attribute of the row, or have the timestamp as a part of the row key, or both.
===== Cell Version Example
The following Put uses a method getCellBuilder() to get a CellBuilder instance
that already has relevant Type and Row set.
[source,java]
----
public static final byte[] CF = "cf".getBytes();
public static final byte[] ATTR = "attr".getBytes();
...
Put put = new Put(Bytes.toBytes(row));
put.add(put.getCellBuilder().setQualifier(ATTR)
.setFamily(CF)
.setValue(Bytes.toBytes(data))
.build());
table.put(put);
----
[[version.delete]]
==== Delete

File diff suppressed because it is too large Load Diff

View File

@ -32,9 +32,6 @@
When should I use HBase?::
See <<arch.overview>> in the Architecture chapter.
Are there other HBase FAQs?::
See the FAQ that is up on the wiki, link:https://wiki.apache.org/hadoop/Hbase/FAQ[HBase Wiki FAQ].
Does HBase support SQL?::
Not really. SQL-ish support for HBase via link:https://hive.apache.org/[Hive] is in development, however Hive is based on MapReduce which is not generally suitable for low-latency requests. See the <<datamodel>> section for examples on the HBase client.

View File

@ -67,18 +67,15 @@ $ tar xzvf hbase-{Version}-bin.tar.gz
$ cd hbase-{Version}/
----
. You are required to set the `JAVA_HOME` environment variable before starting HBase.
You can set the variable via your operating system's usual mechanism, but HBase
provides a central mechanism, _conf/hbase-env.sh_.
Edit this file, uncomment the line starting with `JAVA_HOME`, and set it to the
appropriate location for your operating system.
The `JAVA_HOME` variable should be set to a directory which contains the executable file _bin/java_.
Most modern Linux operating systems provide a mechanism, such as /usr/bin/alternatives on RHEL or CentOS, for transparently switching between versions of executables such as Java.
In this case, you can set `JAVA_HOME` to the directory containing the symbolic link to _bin/java_, which is usually _/usr_.
. You must set the `JAVA_HOME` environment variable before starting HBase.
To make this easier, HBase lets you set it within the _conf/hbase-env.sh_ file. You must locate where Java is
installed on your machine, and one way to find this is by using the _whereis java_ command. Once you have the location,
edit the _conf/hbase-env.sh_ file and uncomment the line starting with _#export JAVA_HOME=_, and then set it to your Java installation path.
+
----
JAVA_HOME=/usr
----
.Example extract from _hbase-env.sh_ where _JAVA_HOME_ is set
# Set environment variables here.
# The java implementation to use.
export JAVA_HOME=/usr/jdk64/jdk1.8.0_112
+
. The _bin/start-hbase.sh_ script is provided as a convenient way to start HBase.
Issue the command, and if all goes well, a message is logged to standard output showing that HBase started successfully.
@ -577,11 +574,11 @@ For more about ZooKeeper configuration, including using an external ZooKeeper in
. Browse to the Web UI.
+
.Web UI Port Changes
NOTE: Web UI Port Changes
+
[NOTE]
====
In HBase newer than 0.98.x, the HTTP ports used by the HBase Web UI changed from 60010 for the
Master and 60030 for each RegionServer to 16010 for the Master and 16030 for the RegionServer.
====
+
If everything is set up correctly, you should be able to connect to the UI for the Master
`http://node-a.example.com:16010/` or the secondary master at `http://node-b.example.com:16010/`

View File

@ -523,7 +523,7 @@ The host name or IP address of the name server (DNS)
+
.Description
Port used by ZooKeeper peers to talk to each other.
See https://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
See https://zookeeper.apache.org/doc/r3.4.10/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
for more information.
+
.Default
@ -535,7 +535,7 @@ Port used by ZooKeeper peers to talk to each other.
+
.Description
Port used by ZooKeeper for leader election.
See https://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
See https://zookeeper.apache.org/doc/r3.4.10/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
for more information.
+
.Default
@ -2027,108 +2027,6 @@ A comma-separated list of
`0`
[[hbase.master.regions.recovery.check.interval]]
*`hbase.master.regions.recovery.check.interval`*::
+
.Description
Regions Recovery Chore interval in milliseconds.
This chore keeps running at this interval to
find all regions with configurable max store file ref count
and reopens them.
+
.Default
`1200000`
[[hbase.regions.recovery.store.file.ref.count]]
*`hbase.regions.recovery.store.file.ref.count`*::
+
.Description
Very large number of ref count on a compacted
store file indicates that it is a ref leak
on that object(compacted store file).
Such files can not be removed after
it is invalidated via compaction.
Only way to recover in such scenario is to
reopen the region which can release
all resources, like the refcount,
leases, etc. This config represents Store files Ref
Count threshold value considered for reopening
regions. Any region with compacted store files
ref count > this value would be eligible for
reopening by master. Here, we get the max
refCount among all refCounts on all
compacted away store files that belong to a
particular region. Default value -1 indicates
this feature is turned off. Only positive
integer value should be provided to
enable this feature.
+
.Default
`-1`
[[hbase.regionserver.slowlog.ringbuffer.size]]
*`hbase.regionserver.slowlog.ringbuffer.size`*::
+
.Description
Default size of ringbuffer to be maintained by each RegionServer in order
to store online slowlog responses. This is an in-memory ring buffer of
requests that were judged to be too slow in addition to the responseTooSlow
logging. The in-memory representation would be complete.
For more details, please look into Doc Section:
<<slow_log_responses, slow_log_responses>>
+
.Default
`256`
[[hbase.regionserver.slowlog.buffer.enabled]]
*`hbase.regionserver.slowlog.buffer.enabled`*::
+
.Description
Indicates whether RegionServers have ring buffer running for storing
Online Slow logs in FIFO manner with limited entries. The size of
the ring buffer is indicated by config: hbase.regionserver.slowlog.ringbuffer.size
The default value is false, turn this on and get latest slowlog
responses with complete data.
For more details, please look into Doc Section:
<<slow_log_responses, slow_log_responses>>
+
.Default
`false`
[[hbase.regionserver.slowlog.systable.enabled]]
*`hbase.regionserver.slowlog.systable.enabled`*::
+
.Description
Should be enabled only if hbase.regionserver.slowlog.buffer.enabled is enabled.
If enabled (true), all slow/large RPC logs would be persisted to system table
hbase:slowlog (in addition to in-memory ring buffer at each RegionServer).
The records are stored in increasing order of time.
Operators can scan the table with various combination of ColumnValueFilter and
time range.
More details are provided in the doc section:
"Get Slow/Large Response Logs from System table hbase:slowlog"
+
.Default
`false`
[[hbase.region.replica.replication.enabled]]
*`hbase.region.replica.replication.enabled`*::
+
@ -2265,6 +2163,121 @@ The percent of region server RPC threads failed to abort RS.
`0`
[[hbase.master.regions.recovery.check.interval]]
*`hbase.master.regions.recovery.check.interval`*::
+
.Description
Regions Recovery Chore interval in milliseconds.
This chore keeps running at this interval to
find all regions with configurable max store file ref count
and reopens them.
+
.Default
`1200000`
[[hbase.regions.recovery.store.file.ref.count]]
*`hbase.regions.recovery.store.file.ref.count`*::
+
.Description
Very large number of ref count on a compacted
store file indicates that it is a ref leak
on that object(compacted store file).
Such files can not be removed after
it is invalidated via compaction.
Only way to recover in such scenario is to
reopen the region which can release
all resources, like the refcount,
leases, etc. This config represents Store files Ref
Count threshold value considered for reopening
regions. Any region with compacted store files
ref count > this value would be eligible for
reopening by master. Here, we get the max
refCount among all refCounts on all
compacted away store files that belong to a
particular region. Default value -1 indicates
this feature is turned off. Only positive
integer value should be provided to
enable this feature.
+
.Default
`-1`
[[hbase.regionserver.slowlog.ringbuffer.size]]
*`hbase.regionserver.slowlog.ringbuffer.size`*::
+
.Description
Default size of ringbuffer to be maintained by each RegionServer in order
to store online slowlog responses. This is an in-memory ring buffer of
requests that were judged to be too slow in addition to the responseTooSlow
logging. The in-memory representation would be complete.
For more details, please look into Doc Section:
<<slow_log_responses, slow_log_responses>>
+
.Default
`256`
[[hbase.regionserver.slowlog.buffer.enabled]]
*`hbase.regionserver.slowlog.buffer.enabled`*::
+
.Description
Indicates whether RegionServers have ring buffer running for storing
Online Slow logs in FIFO manner with limited entries. The size of
the ring buffer is indicated by config: hbase.regionserver.slowlog.ringbuffer.size
The default value is false, turn this on and get latest slowlog
responses with complete data.
For more details, please look into Doc Section:
<<slow_log_responses, slow_log_responses>>
+
.Default
`false`
[[hbase.regionserver.slowlog.systable.enabled]]
*`hbase.regionserver.slowlog.systable.enabled`*::
+
.Description
Should be enabled only if hbase.regionserver.slowlog.buffer.enabled is enabled.
If enabled (true), all slow/large RPC logs would be persisted to system table
hbase:slowlog (in addition to in-memory ring buffer at each RegionServer).
The records are stored in increasing order of time.
Operators can scan the table with various combination of ColumnValueFilter and
time range.
More details are provided in the doc section:
"Get Slow/Large Response Logs from System table hbase:slowlog"
+
.Default
`false`
[[hbase.master.metafixer.max.merge.count]]
*`hbase.master.metafixer.max.merge.count`*::
+
.Description
Maximum regions to merge at a time when we fix overlaps noted in
CJ consistency report, but avoid merging 100 regions in one go!
+
.Default
`64`
[[hbase.rpc.rows.size.threshold.reject]]
*`hbase.rpc.rows.size.threshold.reject`*::
+
@ -2283,3 +2296,4 @@ The percent of region server RPC threads failed to abort RS.
+
.Default
`false`

View File

@ -36,22 +36,15 @@ read and write paths are optimized for values smaller than 100KB in size. When
HBase deals with large numbers of objects over this threshold, referred to here
as medium objects, or MOBs, performance is degraded due to write amplification
caused by splits and compactions. When using MOBs, ideally your objects will be between
100KB and 10MB (see the <<faq>>). HBase ***FIX_VERSION_NUMBER*** adds support
for better managing large numbers of MOBs while maintaining performance,
consistency, and low operational overhead. MOB support is provided by the work
done in link:https://issues.apache.org/jira/browse/HBASE-11339[HBASE-11339]. To
take advantage of MOB, you need to use <<hfilev3,HFile version 3>>. Optionally,
100KB and 10MB (see the <<faq>>). HBase 2 added special internal handling of MOBs
to maintain performance, consistency, and low operational overhead. MOB support is
provided by the work done in link:https://issues.apache.org/jira/browse/HBASE-11339[HBASE-11339].
To take advantage of MOB, you need to use <<hfilev3,HFile version 3>>. Optionally,
configure the MOB file reader's cache settings for each RegionServer (see
<<mob.cache.configure>>), then configure specific columns to hold MOB data.
Client code does not need to change to take advantage of HBase MOB support. The
feature is transparent to the client.
MOB compaction
MOB data is flushed into MOB files after MemStore flush. There will be lots of MOB files
after some time. To reduce MOB file count, there is a periodic task which compacts
small MOB files into a large one (MOB compaction).
=== Configuring Columns for MOB
You can configure columns to support MOB during table creation or alteration,
@ -79,41 +72,6 @@ hcd.setMobThreshold(102400L);
----
====
=== Configure MOB Compaction Policy
By default, MOB files for one specific day are compacted into one large MOB file.
To reduce MOB file count more, there are other MOB Compaction policies supported.
daily policy - compact MOB Files for one day into one large MOB file (default policy)
weekly policy - compact MOB Files for one week into one large MOB file
montly policy - compact MOB Files for one month into one large MOB File
.Configure MOB compaction policy Using HBase Shell
----
hbase> create 't1', {NAME => 'f1', IS_MOB => true, MOB_THRESHOLD => 102400, MOB_COMPACT_PARTITION_POLICY => 'daily'}
hbase> create 't1', {NAME => 'f1', IS_MOB => true, MOB_THRESHOLD => 102400, MOB_COMPACT_PARTITION_POLICY => 'weekly'}
hbase> create 't1', {NAME => 'f1', IS_MOB => true, MOB_THRESHOLD => 102400, MOB_COMPACT_PARTITION_POLICY => 'monthly'}
hbase> alter 't1', {NAME => 'f1', IS_MOB => true, MOB_THRESHOLD => 102400, MOB_COMPACT_PARTITION_POLICY => 'daily'}
hbase> alter 't1', {NAME => 'f1', IS_MOB => true, MOB_THRESHOLD => 102400, MOB_COMPACT_PARTITION_POLICY => 'weekly'}
hbase> alter 't1', {NAME => 'f1', IS_MOB => true, MOB_THRESHOLD => 102400, MOB_COMPACT_PARTITION_POLICY => 'monthly'}
----
=== Configure MOB Compaction mergeable threshold
If the size of a mob file is less than this value, it's regarded as a small file and needs to
be merged in mob compaction. The default value is 1280MB.
====
[source,xml]
----
<property>
<name>hbase.mob.compaction.mergeable.threshold</name>
<value>10000000000</value>
</property>
----
====
=== Testing MOB
The utility `org.apache.hadoop.hbase.IntegrationTestIngestWithMOB` is provided to assist with testing
@ -133,9 +91,219 @@ $ sudo -u hbase hbase org.apache.hadoop.hbase.IntegrationTestIngestWithMOB \
* `*maxMobDataSize*` is the maximum value for the size of MOB data.
The default is 5 kB, expressed in bytes.
=== MOB architecture
This section is derived from information found in
link:https://issues.apache.org/jira/browse/HBASE-11339[HBASE-11339], which covered the initial GA
implementation of MOB in HBase and
link:https://issues.apache.org/jira/browse/HBASE-22749[HBASE-22749], which improved things by
parallelizing MOB maintenance across the RegionServers. For more information see
the last version of the design doc created during the initial work,
"link:https://github.com/apache/hbase/blob/master/dev-support/design-docs/HBASE-11339%20MOB%20GA%20design.pdf[HBASE-11339 MOB GA design.pdf]",
and the design doc for the distributed mob compaction feature,
"link:https://github.com/apache/hbase/blob/master/dev-support/design-docs/HBASE-22749%20MOB%20distributed%20compaction.pdf[HBASE-22749 MOB distributed compaction.pdf]".
==== Overview
The MOB feature reduces the overall IO load for configured column families by storing values that
are larger than the configured threshold outside of the normal regions to avoid splits, merges, and
most importantly normal compactions.
When a cell is first written to a region it is stored in the WAL and memstore regardless of value
size. When memstores from a column family configured to use MOB are eventually flushed two hfiles
are written simultaneously. Cells with a value smaller than the threshold size are written to a
normal region hfile. Cells with a value larger than the threshold are written into a special MOB
hfile and also have a MOB reference cell written into the normal region HFile. As the Region Server
flushes a MOB enabled memstore and closes a given normal region HFile it appends metadata that lists
each of the special MOB hfiles referenced by the cells within.
MOB reference cells have the same key as the cell they are based on. The value of the reference cell
is made up of two pieces of metadata: the size of the actual value and the MOB hfile that contains
the original cell. In addition to any tags originally written to HBase, the reference cell prepends
two additional tags. The first is a marker tag that says the cell is a MOB reference. This can be
used later to scan specifically just for reference cells. The second stores the namespace and table
at the time the MOB hfile is written out. This tag is used to optimize how the MOB system finds
the underlying value in MOB hfiles after a series of HBase snapshot operations (ref HBASE-12332).
Note that tags are only available within HBase servers and by default are not sent over RPCs.
All MOB hfiles for a given table are managed within a logical region that does not directly serve
requests. When these MOB hfiles are created from a flush or MOB compaction they are placed in a
dedicated mob data area under the hbase root directory specific to the namespace, table, mob
logical region, and column family. In general that means a path structured like:
----
%HBase Root Dir%/mobdir/data/%namespace%/%table%/%logical region%/%column family%/
----
With default configs, an example table named 'some_table' in the
default namespace with a MOB enabled column family named 'foo' this HDFS directory would be
----
/hbase/mobdir/data/default/some_table/372c1b27e3dc0b56c3a031926e5efbe9/foo/
----
These MOB hfiles are maintained by special chores in the HBase Master and across the individual
Region Servers. Specifically those chores take care of enforcing TTLs and compacting them. Note that
this compaction is primarily a matter of controlling the total number of files in HDFS because our
operational assumptions for MOB data is that it will seldom update or delete.
When a given MOB hfile is no longer needed as a result of our compaction process then a chore in
the Master will take care of moving it to the archive just
like any normal hfile. Because the table's mob region is independent of all the normal regions it
can coexist with them in the regular archive storage area:
----
/hbase/archive/data/default/some_table/372c1b27e3dc0b56c3a031926e5efbe9/foo/
----
The same hfile cleaning chores that take care of eventually deleting unneeded archived files from
normal regions thus also will take care of these MOB hfiles. As such, if there is a snapshot of a
MOB enabled table then the cleaning system will make sure those MOB files stick around in the
archive area as long as they are needed by a snapshot or a clone of a snapshot.
==== MOB compaction
Each time the memstore for a MOB enabled column family performs a flush HBase will write values over
the MOB threshold into MOB specific hfiles. When normal region compaction occurs the Region Server
rewrites the normal data files while maintaining references to these MOB files without rewriting
them. Normal client lookups for MOB values transparently will receive the original values because
the Region Server internals take care of using the reference data to then pull the value out of a
specific MOB file. This indirection means that building up a large number of MOB hfiles doesn't
impact the overall time to retrieve any specific MOB cell. Thus, we need not perform compactions of
the MOB hfiles nearly as often as normal hfiles. As a result, HBase saves IO by not rewriting MOB
hfiles as a part of the periodic compactions a Region Server does on its own.
However, if deletes and updates of MOB cells are frequent then this indirection will begin to waste
space. The only way to stop using the space of a particular MOB hfile is to ensure no cells still
hold references to it. To do that we need to ensure we have written the current values into a new
MOB hfile. If our backing filesystem has a limitation on the number of files that can be present, as
HDFS does, then even if we do not have deletes or updates of MOB cells eventually there will be a
sufficient number of MOB hfiles that we will need to coallesce them.
Periodically a chore in the master coordinates having the region servers
perform a special major compaction that also handles rewritting new MOB files. Like all compactions
the Region Server will create updated hfiles that hold both the cells that are smaller than the MOB
threshold and cells that hold references to the newly rewritten MOB file. Because this rewriting has
the advantage of looking across all active cells for the region our several small MOB files should
end up as a single MOB file per region. The chore defaults to running weekly and can be
configured by setting `hbase.mob.compaction.chore.period` to the desired period in seconds.
====
[source,xml]
----
<property>
<name>hbase.mob.compaction.chore.period</name>
<value>2592000</value>
<description>Example of changing the chore period from a week to a month.</description>
</property>
----
====
By default, the periodic MOB compaction coordination chore will attempt to keep every region
busy doing compactions in parallel in order to maximize the amount of work done on the cluster.
If you need to tune the amount of IO this compaction generates on the underlying filesystem, you
can control how many concurrent region-level compaction requests are allowed by setting
`hbase.mob.major.compaction.region.batch.size` to an integer number greater than zero. If you set
the configuration to 0 then you will get the default behavior of attempting to do all regions in
parallel.
====
[source,xml]
----
<property>
<name>hbase.mob.major.compaction.region.batch.size</name>
<value>1</value>
<description>Example of switching from "as parallel as possible" to "serially"</description>
</property>
----
====
==== MOB file archiving
Eventually we will have MOB hfiles that are no longer needed. Either clients will overwrite the
value or a MOB-rewriting compaction will store a reference to a newer larger MOB hfile. Because any
given MOB cell could have originally been written either in the current region or in a parent region
that existed at some prior point in time, individual Region Servers do not decide when it is time
to archive MOB hfiles. Instead a periodic chore in the Master evaluates MOB hfiles for archiving.
A MOB HFile will be subject to archiving under any of the following conditions:
* Any MOB HFile older than the column family's TTL
* Any MOB HFile older than a "too recent" threshold with no references to it from the regular hfiles
for all regions in a column family
To determine if a MOB HFile meets the second criteria the chore extracts metadata from the regular
HFiles for each MOB enabled column family for a given table. That metadata enumerates the complete
set of MOB HFiles needed to satisfy the references stored in the normal HFile area.
The period of the cleaner chore can be configued by setting `hbase.master.mob.cleaner.period` to a
positive integer number of seconds. It defaults to running daily. You should not need to tune it
unless you have a very aggressive TTL or a very high rate of MOB updates with a correspondingly
high rate of non-MOB compactions.
=== MOB Optimization Tasks
==== Further limiting write amplification
If your MOB workload has few to no updates or deletes then you can opt-in to MOB compactions that
optimize for limiting the amount of write amplification. It acheives this by setting a
size threshold to ignore MOB files during the compaction process. When a given region goes
through MOB compaction it will evaluate the size of the MOB file that currently holds the actual
value and skip rewriting the value if that file is over threshold.
The bound of write amplification in this mode can be approximated as
stem:["Write Amplification" = log_K(M/S)] where *K* is the number of files in compaction
selection, *M* is the configurable threshold for MOB files size, and *S* is the minmum size of
memstore flushes that create MOB files in the first place. For example given 5 files picked up per
compaction, a threshold of 1 GB, and a flush size of 10MB the write amplification will be
stem:[log_5((1GB)/(10MB)) = log_5(100) = 2.86].
If we are using an underlying filesystem with a limitation on the number of files, such as HDFS,
and we know our expected data set size we can choose our maximum file size in order to approach
this limit but stay within it in order to minimize write amplification. For example, if we expect to
store a petabyte and we have a conservative limitation of a million files in our HDFS instance, then
stem:[(1PB)/(1M) = 1GB] gives us a target limitation of a gigabyte per MOB file.
To opt-in to this compaction mode you must set `hbase.mob.compaction.type` to `optimized`. The
default MOB size threshold in this mode is set to 1GB. It can be changed by setting
`hbase.mob.compactions.max.file.size` to a positive integer number of bytes.
====
[source,xml]
----
<property>
<name>hbase.mob.compaction.type</name>
<value>optimized</value>
<description>opt-in to write amplification optimized mob compaction.</description>
</property>
<property>
<name>hbase.mob.compactions.max.file.size</name>
<value>10737418240</value>
<description>Example of tuning the max mob file size to 10GB</dscription>
</property>
----
====
Additionally, when operating in this mode the compaction process will seek to avoid writing MOB
files that are over the max file threshold. As it is writing out a additional MOB values into a MOB
hfile it will check to see if the additional data causes the hfile to be over the max file size.
When the hfile of MOB values reaches limit, the MOB hfile is committed to the MOB storage area and
a new one is created. The hfile with reference cells will track the complete set of MOB hfiles it
needs in its metadata.
.Be mindful of total time to complete compaction of a region
[WARNING]
====
When using the write amplification optimized compaction mode you need to watch for the maximum time
to compact a single region. If it nears an hour you should read through the troubleshooting section
below <<mob.troubleshoot.cleaner.toonew>>. Failure to make the adjustments discussed there could
lead to dataloss.
====
[[mob.cache.configure]]
=== Configuring the MOB Cache
==== Configuring the MOB Cache
Because there can be a large number of MOB files at any time, as compared to the number of HFiles,
@ -181,85 +349,61 @@ suit your environment, and restart or rolling restart the RegionServer.
----
====
=== MOB Optimization Tasks
==== Manually Compacting MOB Files
To manually compact MOB files, rather than waiting for the
<<mob.cache.configure,configuration>> to trigger compaction, use the
`compact` or `major_compact` HBase shell commands. These commands
periodic chore to trigger compaction, use the
`major_compact` HBase shell commands. These commands
require the first argument to be the table name, and take a column
family as the second argument. and take a compaction type as the third argument.
family as the second argument. If used with a column family that includes MOB data, then
these operator requests will result in the MOB data being compacted.
----
hbase> compact 't1', 'c1, MOB
hbase> major_compact 't1', 'c1, MOB
hbase> major_compact 't1'
hbase> major_compact 't2', 'c1
----
These commands are also available via `Admin.compact` and
`Admin.majorCompact` methods.
=== MOB architecture
This section is derived from information found in
link:https://issues.apache.org/jira/browse/HBASE-11339[HBASE-11339]. For more information see
the attachment on that issue
"link:https://issues.apache.org/jira/secure/attachment/12724468/HBase%20MOB%20Design-v5.pdf[Base MOB Design-v5.pdf]".
==== Overview
The MOB feature reduces the overall IO load for configured column families by storing values that
are larger than the configured threshold outside of the normal regions to avoid splits, merges, and
most importantly normal compactions.
When a cell is first written to a region it is stored in the WAL and memstore regardless of value
size. When memstores from a column family configured to use MOB are eventually flushed two hfiles
are written simultaneously. Cells with a value smaller than the threshold size are written to a
normal region hfile. Cells with a value larger than the threshold are written into a special MOB
hfile and also have a MOB reference cell written into the normal region HFile.
MOB reference cells have the same key as the cell they are based on. The value of the reference cell
is made up of two pieces of metadata: the size of the actual value and the MOB hfile that contains
the original cell. In addition to any tags originally written to HBase, the reference cell prepends
two additional tags. The first is a marker tag that says the cell is a MOB reference. This can be
used later to scan specifically just for reference cells. The second stores the namespace and table
at the time the MOB hfile is written out. This tag is used to optimize how the MOB system finds
the underlying value in MOB hfiles after a series of HBase snapshot operations (ref HBASE-12332).
Note that tags are only available within HBase servers and by default are not sent over RPCs.
All MOB hfiles for a given table are managed within a logical region that does not directly serve
requests. When these MOB hfiles are created from a flush or MOB compaction they are placed in a
dedicated mob data area under the hbase root directory specific to the namespace, table, mob
logical region, and column family. In general that means a path structured like:
----
%HBase Root Dir%/mobdir/data/%namespace%/%table%/%logical region%/%column family%/
----
With default configs, an example table named 'some_table' in the
default namespace with a MOB enabled column family named 'foo' this HDFS directory would be
----
/hbase/mobdir/data/default/some_table/372c1b27e3dc0b56c3a031926e5efbe9/foo/
----
These MOB hfiles are maintained by special chores in the HBase Master rather than by any individual
Region Server. Specifically those chores take care of enforcing TTLs and compacting them. Note that
this compaction is primarily a matter of controlling the total number of files in HDFS because our
operational assumptions for MOB data is that it will seldom update or delete.
When a given MOB hfile is no longer needed as a result of our compaction process it is archived just
like any normal hfile. Because the table's mob region is independent of all the normal regions it
can coexist with them in the regular archive storage area:
----
/hbase/archive/data/default/some_table/372c1b27e3dc0b56c3a031926e5efbe9/foo/
----
The same hfile cleaning chores that take care of eventually deleting unneeded archived files from
normal regions thus also will take care of these MOB hfiles.
This same request can be made via the `Admin.majorCompact` Java API.
=== MOB Troubleshooting
[[mob.troubleshoot.cleaner.toonew]]
==== Adjusting the MOB cleaner's tolerance for new hfiles
The MOB cleaner chore ignores all MOB hfiles that were created more recently than an hour prior to
the start of the chore to ensure we don't miss the reference metadata from the corresponding regular
hfile. Without this safety check it would be possible for the cleaner chore to see a MOB hfile for
an in progress flush or compaction and prematurely archive the MOB data. This default buffer should
be sufficient for normal use.
You will need to adjust the tolerance if you use write amplification optimized MOB compaction and
the combination of your underlying filesystem performance and data shape is such that it could take
more than an hour to complete major compaction of a single region. For example, if your MOB data is
distributed such that your largest region adds 80GB of MOB data between compactions that include
rewriting MOB data and your HDFS cluster is only capable of writing 20MB/s for a single file then
when performing the optimized compaction the Region Server will take about a minute to write the
first 1GB MOB hfile and then another hour and seven minutes to write the remaining seventy-nine 1GB
MOB hfiles before finally committing the new reference hfile at the end of the compaction. Given
this example, you would need a larger tolerance window.
You will also need to adjust the tolerance if Region Server flush operations take longer than an
hour for the two HDFS move operations needed to commit both the MOB hfile and the normal hfile that
references it. Such a delay should not happen with a normally configured and healthy HDFS and HBase.
The cleaner's window for "too recent" is controlled by setting `hbase.mob.min.age.archive` to a
positive integer number of milliseconds.
====
[source,xml]
----
<property>
<name>hbase.mob.min.age.archive</name>
<value>86400000</value>
<description>Example of tuning the cleaner to only archive files older than a day.</dscription>
</property>
----
====
==== Retrieving MOB metadata through the HBase Shell
While working on troubleshooting failures in the MOB system you can retrieve some of the internal
@ -468,3 +612,64 @@ $ hdfs dfs -count /hbase/mobdir/data/default/some_table
+
This data is spurious and may be reclaimed. You should sideline it, verify your applications view
of the table, and then delete it.
=== MOB Upgrade Considerations
Generally, data stored using the MOB feature should transparently continue to work correctly across
HBase upgrades.
==== Upgrading to a version with the "distributed MOB compaction" feature
Prior to the work in HBASE-22749, "Distributed MOB compactions", HBase had the Master coordinate all
compaction maintenance of the MOB hfiles. Centralizing management of the MOB data allowed for space
optimizations but safely coordinating that managemet with Region Servers resulted in edge cases that
caused data loss (ref link:https://issues.apache.org/jira/browse/HBASE-22075[HBASE-22075]).
Users of the MOB feature upgrading to a version of HBase that includes HBASE-22749 should be aware
of the following changes:
* The MOB system no longer allows setting "MOB Compaction Policies"
* The MOB system no longer attempts to group MOB values by the date of the original cell's timestamp
according to said compaction policies, daily or otherwise
* The MOB system no longer needs to track individual cell deletes through the use of special
files in the MOB storage area with the suffix `_del`. After upgrading you should sideline these
files.
* Under default configuration the MOB system should take much less time to perform a compaction of
MOB stored values. This is a direct consequence of the fact that HBase will place a much larger
load on the underlying filesystem when doing compactions of MOB stored values; the additional load
should be a multiple on the order of magnitude of number of region servers. I.e. for a cluster
with three region servers and two masters the default configuration should have HBase put three
times the load on HDFS during major compactions that rewrite MOB data when compared to Master
handled MOB compaction; it should also be approximately three times as fast.
* When the MOB system detects that a table has hfiles with references to MOB data but the reference
hfiles do not yet have the needed file level metadata (i.e. from use of the MOB feature prior to
HBASE-22749) then it will refuse to archive _any_ MOB hfiles from that table. The normal course of
periodic compactions done by Region Servers will update existing hfiles with MOB references, but
until a given table has been through the needed compactions operators should expect to see an
increased amount of storage used by the MOB feature.
* Performing a compaction with type "MOB" no longer has special handling to compact specifically the
MOB hfiles. Instead it will issue a warning and do a compaction of the table. For example using
the HBase shell as follows will result in a warning in the Master logs followed by a major
compaction of the 'example' table in its entirety or for the 'big' column respectively.
+
----
hbase> major_compact 'example', nil, 'MOB'
hbase> major_compact 'example', 'big', 'MOB'
----
+
The same is true for directly using the Java API for
`admin.majorCompact(TableName.valueOf("example"), CompactType.MOB)`.
* Similarly, manually performing a major compaction on a table or region will also handle compacting
the MOB stored values for that table or region respectively.
The following configuration setting has been deprecated and replaced:
* `hbase.master.mob.ttl.cleaner.period` has been replaced with `hbase.master.mob.cleaner.period`
The following configuration settings are no longer used:
* `hbase.mob.compaction.mergeable.threshold`
* `hbase.mob.delfile.max.count`
* `hbase.mob.compaction.batch.size`
* `hbase.mob.compactor.class`
* `hbase.mob.compaction.threads.max`

View File

@ -0,0 +1,269 @@
////
/**
*
* 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.
*/
////
[[hbtop]]
= hbtop
:doctype: book
:numbered:
:toc: left
:icons: font
:experimental:
== Overview
`hbtop` is a real-time monitoring tool for HBase like Unix's top command.
It can display summary information as well as metrics per Region/Namespace/Table/RegionServer.
In this tool, you can see the metrics sorted by a selected field and filter the metrics to see only metrics you really want to see.
Also, with the drill-down feature, you can find hot regions easily in a top-down manner.
== Usage
You can run hbtop with the following command:
----
$ hbase hbtop
----
In this case, the values of `hbase.client.zookeeper.quorum` and `zookeeper.znode.parent` in `hbase-site.xml` in the classpath or the default values of them are used to connect.
Or, you can specify your own zookeeper quorum and znode parent as follows:
----
$ hbase hbtop -Dhbase.client.zookeeper.quorum=<zookeeper quorum> -Dzookeeper.znode.parent=<znode parent>
----
image::https://hbase.apache.org/hbtop-images/top_screen.gif[Top screen]
The top screen consists of a summary part and of a metrics part.
In the summary part, you can see `HBase Version`, `Cluster ID`, `The number of region servers`, `Region count`, `Average Cluster Load` and `Aggregated Request/s`.
In the metrics part, you can see metrics per Region/Namespace/Table/RegionServer depending on the selected mode.
The top screen is refreshed in a certain period 3 seconds by default.
=== Scrolling metric records
You can scroll the metric records in the metrics part.
image::https://hbase.apache.org/hbtop-images/scrolling_metric_records.gif[Scrolling metric records]
=== Command line arguments
[options="header"]
|=================================
| Argument | Description
| -d,--delay &lt;arg&gt; | The refresh delay (in seconds); default is 3 seconds
| -h,--help | Print usage; for help while the tool is running press `h` key
| -m,--mode &lt;arg&gt; | The mode; `n` (Namespace)&#124;`t` (Table)&#124;r (Region)&#124;`s` (RegionServer), default is `r` (Region)
|=================================
=== Modes
There are the following 4 modes in hbtop:
[options="header"]
|=================================
| Mode | Description
| Region | Showing metric records per region
| Namespace | Showing metric records per namespace
| Table | Showing metric records per table
| RegionServer | Showing metric records per region server
|=================================
==== Region mode
In Region mode, the default sort field is `#REQ/S`.
The fields in this mode are as follows:
[options="header"]
|=================================
| Field | Description | Displayed by default
| RNAME | Region Name | false
| NAMESPACE | Namespace Name | true
| TABLE | Table Name | true
| SCODE | Start Code | false
| REPID | Replica ID | false
| REGION | Encoded Region Name | true
| RS | Short Region Server Name | true
| LRS | Long Region Server Name | false
| #REQ/S | Request Count per second | true
| #READ/S | Read Request Count per second | true
| #FREAD/S | Filtered Read Request Count per second | true
| #WRITE/S | Write Request Count per second | true
| SF | StoreFile Size | true
| USF | Uncompressed StoreFile Size | false
| #SF | Number of StoreFiles | true
| MEMSTORE | MemStore Size | true
| LOCALITY | Block Locality | true
| SKEY | Start Key | false
| #COMPingCELL | Compacting Cell Count | false
| #COMPedCELL | Compacted Cell Count | false
| %COMP | Compaction Progress | false
| LASTMCOMP | Last Major Compaction Time | false
|=================================
==== Namespace mode
In Namespace mode, the default sort field is `#REQ/S`.
The fields in this mode are as follows:
[options="header"]
|=================================
| Field | Description | Displayed by default
| NAMESPACE | Namespace Name | true
| #REGION | Region Count | true
| #REQ/S | Request Count per second | true
| #READ/S | Read Request Count per second | true
| #FREAD/S | Filtered Read Request Count per second | true
| #WRITE/S | Write Request Count per second | true
| SF | StoreFile Size | true
| USF | Uncompressed StoreFile Size | false
| #SF | Number of StoreFiles | true
| MEMSTORE | MemStore Size | true
|=================================
==== Table mode
In Table mode, the default sort field is `#REQ/S`.
The fields in this mode are as follows:
[options="header"]
|=================================
| Field | Description | Displayed by default
| NAMESPACE | Namespace Name | true
| TABLE | Table Name | true
| #REGION | Region Count | true
| #REQ/S | Request Count per second | true
| #READ/S | Read Request Count per second | true
| #FREAD/S | Filtered Read Request Count per second | true
| #WRITE/S | Write Request Count per second | true
| SF | StoreFile Size | true
| USF | Uncompressed StoreFile Size | false
| #SF | Number of StoreFiles | true
| MEMSTORE | MemStore Size | true
|=================================
==== RegionServer mode
In RegionServer mode, the default sort field is `#REQ/S`.
The fields in this mode are as follows:
[options="header"]
|=================================
| Field | Description | Displayed by default
| RS | Short Region Server Name | true
| LRS | Long Region Server Name | false
| #REGION | Region Count | true
| #REQ/S | Request Count per second | true
| #READ/S | Read Request Count per second | true
| #FREAD/S | Filtered Read Request Count per second | true
| #WRITE/S | Write Request Count per second | true
| SF | StoreFile Size | true
| USF | Uncompressed StoreFile Size | false
| #SF | Number of StoreFiles | true
| MEMSTORE | MemStore Size | true
| UHEAP | Used Heap Size | true
| MHEAP | Max Heap Size | true
|=================================
=== Changing mode
You can change mode by pressing `m` key in the top screen.
image::https://hbase.apache.org/hbtop-images/changing_mode.gif[Changing mode]
=== Changing the refresh delay
You can change the refresh by pressing `d` key in the top screen.
image::https://hbase.apache.org/hbtop-images/changing_refresh_delay.gif[Changing the refresh delay]
=== Changing the displayed fields
You can move to the field screen by pressing `f` key in the top screen. In the fields screen, you can change the displayed fields by choosing a field and pressing `d` key or `space` key.
image::https://hbase.apache.org/hbtop-images/changing_displayed_fields.gif[Changing the displayed fields]
=== Changing the sort field
You can move to the fields screen by pressing `f` key in the top screen. In the field screen, you can change the sort field by choosing a field and pressing `s`. Also, you can change the sort order (ascending or descending) by pressing `R` key.
image::https://hbase.apache.org/hbtop-images/changing_sort_field.gif[Changing the sort field]
=== Changing the order of the fields
You can move to the fields screen by pressing `f` key in the top screen. In the field screen, you can change the order of the fields.
image::https://hbase.apache.org/hbtop-images/changing_order_of_fields.gif[Changing the sort field]
=== Filters
You can filter the metric records with the filter feature. We can add filters by pressing `o` key for ignoring case or `O` key for case sensitive.
image::https://hbase.apache.org/hbtop-images/adding_filters.gif[Adding filters]
The syntax is as follows:
----
<Field><Operator><Value>
----
For example, we can add filters like the following:
----
NAMESPACE==default
REQ/S>1000
----
The operators we can specify are as follows:
[options="header"]
|=================================
| Operator | Description
| = | Partial match
| == | Exact match
| > | Greater than
| >= | Greater than or equal to
| < | Less than
| <= | Less than and equal to
|=================================
You can see the current filters by pressing `^o` key and clear them by pressing `=` key.
image::https://hbase.apache.org/hbtop-images/showing_and_clearing_filters.gif[Showing and clearing filters]
=== Drilling down
You can drill down the metric record by choosing a metric record that you want to drill down and pressing `i` key in the top screen. With this feature, you can find hot regions easily in a top-down manner.
image::https://hbase.apache.org/hbtop-images/driling_down.gif[Drilling down]
=== Help screen
You can see the help screen by pressing `h` key in the top screen.
image::https://hbase.apache.org/hbtop-images/help_screen.gif[Help screen]
== Others
=== How hbtop gets the metrics data
hbtop gets the metrics from ClusterMetrics which is returned as the result of a call to Admin#getClusterMetrics() on the current HMaster. To add metrics to hbtop, they will need to be exposed via ClusterMetrics.

View File

@ -59,11 +59,9 @@ attribute can have one of four values.
* _EAGER_: This is _BASIC_ policy plus in-memory compaction of flushes (much like the on-disk compactions done to hfiles); on compaction we apply on-disk rules eliminating versions, duplicates, ttl'd cells, etc.
* _ADAPTIVE_: Adaptive compaction adapts to the workload. It applies either index compaction or data compaction based on the ratio of duplicate cells in the data. Experimental.
To enable _BASIC_ on the _info_ column family in the table _radish_, disable the table and add the attribute to the _info_ column family, and then reenable:
To enable _BASIC_ on the _info_ column family in the table _radish_, add the attribute to the _info_ column family:
[source,ruby]
----
hbase(main):002:0> disable 'radish'
Took 0.5570 seconds
hbase(main):003:0> alter 'radish', {NAME => 'info', IN_MEMORY_COMPACTION => 'BASIC'}
Updating all regions with the new schema...
All regions updated.
@ -77,8 +75,6 @@ COLUMN FAMILIES DESCRIPTION
'IN_MEMORY_COMPACTION' => 'BASIC'}}
1 row(s)
Took 0.0239 seconds
hbase(main):005:0> enable 'radish'
Took 0.7537 seconds
----
Note how the IN_MEMORY_COMPACTION attribute shows as part of the _METADATA_ map.

View File

@ -120,7 +120,7 @@ You might find the more selective `hbase mapredcp` tool output of interest; it l
to run a basic mapreduce job against an hbase install. It does not include configuration. You'll probably need to add
these if you want your MapReduce job to find the target cluster. You'll probably have to also add pointers to extra jars
once you start to do anything of substance. Just specify the extras by passing the system propery `-Dtmpjars` when
you run `hbase mapredcp`.
you run `hbase mapredcp`.
For jobs that do not package their dependencies or call `TableMapReduceUtil#addDependencyJars`, the following command structure is necessary:
@ -417,8 +417,8 @@ public static class MyMapper extends TableMapper<ImmutableBytesWritable, Put> {
private static Put resultToPut(ImmutableBytesWritable key, Result result) throws IOException {
Put put = new Put(key.get());
for (KeyValue kv : result.raw()) {
put.add(kv);
for (Cell cell : result.listCells()) {
put.add(cell);
}
return put;
}

View File

@ -46,29 +46,41 @@ image::offheap-overview.png[]
== Offheap read-path
In HBase-2.0.0, link:https://issues.apache.org/jira/browse/HBASE-11425[HBASE-11425] changed the HBase read path so it
could hold the read-data off-heap (from BucketCache) avoiding copying of cached data on to the java heap.
This reduces GC pauses given there is less garbage made and so less to clear. The off-heap read path has a performance
that is similar/better to that of the on-heap LRU cache. This feature is available since HBase 2.0.0.
If the BucketCache is in `file` mode, fetching will always be slower compared to the native on-heap LruBlockCache.
This reduces GC pauses given there is less garbage made and so less to clear. The off-heap read path can have a performance
that is similar or better to that of the on-heap LRU cache. This feature is available since HBase 2.0.0.
Refer to below blogs for more details and test results on off heaped read path
link:https://blogs.apache.org/hbase/entry/offheaping_the_read_path_in[Offheaping the Read Path in Apache HBase: Part 1 of 2]
and link:https://blogs.apache.org/hbase/entry/offheap-read-path-in-production[Offheap Read-Path in Production - The Alibaba story]
For an end-to-end off-heaped read-path, first of all there should be an off-heap backed <<offheap.blockcache, BucketCache>>. Configure 'hbase.bucketcache.ioengine' to off-heap in
_hbase-site.xml_. Also specify the total capacity of the BucketCache using `hbase.bucketcache.size` config. Please remember to adjust value of 'HBASE_OFFHEAPSIZE' in
_hbase-env.sh_. This is how we specify the max possible off-heap memory allocation for the RegionServer java process.
This should be bigger than the off-heap BC size. Please keep in mind that there is no default for `hbase.bucketcache.ioengine`
which means the BC is turned OFF by default (See <<direct.memory, Direct Memory Usage In HBase>>).
For an end-to-end off-heaped read-path, all you have to do is enable an off-heap backed <<offheap.blockcache>>(BC).
Configure _hbase.bucketcache.ioengine_ to be _offheap_ in _hbase-site.xml_ (See <<bc.deploy.modes>> to learn more about _hbase.bucketcache.ioengine_ options).
Also specify the total capacity of the BC using `hbase.bucketcache.size` config. Please remember to adjust value of 'HBASE_OFFHEAPSIZE' in
_hbase-env.sh_ (See <<bc.example>> for help sizing and an example enabling). This configuration is for specifying the maximum
possible off-heap memory allocation for the RegionServer java process. This should be bigger than the off-heap BC size
to accommodate usage by other features making use of off-heap memory such as Server RPC buffer pool and short-circuit
reads (See discussion in <<bc.example>>).
Next thing to tune is the ByteBuffer pool on the RPC server side:
Please keep in mind that there is no default for `hbase.bucketcache.ioengine`
which means the BC is OFF by default (See <<direct.memory>>).
This is all you need to do to enable off-heap read path. Most buffers in HBase are already off-heap. With BC off-heap,
the read pipeline will copy data between HDFS and the server socket send of the results back to the client.
[[regionserver.offheap.rpc.bb.tuning]]
===== Tuning the RPC buffer pool
It is possible to tune the ByteBuffer pool on the RPC server side
used to accumulate the cell bytes and create result cell blocks to send back to the client side.
`hbase.ipc.server.reservoir.enabled` can be used to turn this pool ON or OFF. By default this pool is ON and available. HBase will create off-heap ByteBuffers
and pool them them by default. Please make sure not to turn this OFF if you want end-to-end off-heaping in read path.
NOTE: the config keys which start with prefix `hbase.ipc.server.reservoir` are deprecated in HBase3.x. If you are still
in HBase2.x, then just use the old config keys. otherwise if in HBase3.x, please use the new config keys.
(See <<regionserver.read.hdfs.block.offheap,deprecated and new configs in HBase3.x>>)
The buffers from this pool will be used to accumulate the cell bytes and create a result cell block to send back to the client side.
`hbase.ipc.server.reservoir.enabled` can be used to turn this pool ON or OFF. By default this pool is ON and available. HBase will create off heap ByteBuffers
and pool them. Please make sure not to turn this OFF if you want end-to-end off-heaping in read path.
If this pool is turned off, the server will create temp buffers on heap to accumulate the cell bytes and make a result cell block. This can impact the GC on a highly read loaded server.
If this pool is turned off, the server will create temp buffers on heap to accumulate the cell bytes and
make a result cell block. This can impact the GC on a highly read loaded server.
Next thing to tune is the ByteBuffer pool on the RPC server side:
The user can tune this pool with respect to how many buffers are in the pool and what should be the size of each ByteBuffer.
Use the config `hbase.ipc.server.reservoir.initial.buffer.size` to tune each of the buffer sizes. Default is 64 KB for HBase2.x, while it will be changed to 65KB by default for HBase3.x
(see link:https://issues.apache.org/jira/browse/HBASE-22532[HBASE-22532])

View File

@ -51,7 +51,8 @@ Options:
Commands:
Some commands take arguments. Pass no args or -h for usage.
shell Run the HBase shell
hbck Run the hbase 'fsck' tool
hbck Run the HBase 'fsck' tool. Defaults read-only hbck1.
Pass '-j /path/to/HBCK2.jar' to run hbase-2.x HBCK2.
snapshot Tool for managing snapshots
wal Write-ahead-log analyzer
hfile Store file analyzer
@ -386,12 +387,33 @@ Each command except `RowCounter` and `CellCounter` accept a single `--help` argu
[[hbck]]
=== HBase `hbck`
To run `hbck` against your HBase cluster run `$./bin/hbase hbck`. At the end of the command's output it prints `OK` or `INCONSISTENCY`.
If your cluster reports inconsistencies, pass `-details` to see more detail emitted.
If inconsistencies, run `hbck` a few times because the inconsistency may be transient (e.g. cluster is starting up or a region is splitting).
Passing `-fix` may correct the inconsistency (This is an experimental feature).
The `hbck` tool that shipped with hbase-1.x has been made read-only in hbase-2.x. It is not able to repair
hbase-2.x clusters as hbase internals have changed. Nor should its assessments in read-only mode be
trusted as it does not understand hbase-2.x operation.
For more information, see <<hbck.in.depth>>.
A new tool, <<HBCK2>>, described in the next section, replaces `hbck`.
[[HBCK2]]
=== HBase `HBCK2`
`HBCK2` is the successor to <<hbck>>, the hbase-1.x fix tool (A.K.A `hbck1`). Use it in place of `hbck1`
making repairs against hbase-2.x installs.
`HBCK2` does not ship as part of hbase. It can be found as a subproject of the companion
link:https://github.com/apache/hbase-operator-tools[hbase-operator-tools] repository at
link:https://github.com/apache/hbase-operator-tools/tree/master/hbase-hbck2[Apache HBase HBCK2 Tool].
`HBCK2` was moved out of hbase so it could evolve at a cadence apart from that of hbase core.
See the [https://github.com/apache/hbase-operator-tools/tree/master/hbase-hbck2](HBCK2) Home Page
for how `HBCK2` differs from `hbck1`, and for how to build and use it.
Once built, you can run `HBCK2` as follows:
```
$ hbase hbck -j /path/to/HBCK2.jar
```
This will generate `HBCK2` usage describing commands and options.
[[hfile_tool2]]
=== HFile Tool
@ -399,6 +421,8 @@ For more information, see <<hbck.in.depth>>.
See <<hfile_tool>>.
=== WAL Tools
For bulk replaying WAL files or _recovered.edits_ files, see
<<walplayer>>. For reading/verifying individual files, read on.
[[hlog_tool]]
==== FSHLog tool
@ -506,6 +530,13 @@ Caching for the input Scan is configured via `hbase.client.scanner.caching`
By default, CopyTable utility only copies the latest version of row cells unless `--versions=n` is explicitly specified in the command.
====
.Data Load
[NOTE]
====
CopyTable does not perform a diff, it copies all Cells in between the specified startrow/stoprow starttime/endtime range.
This means that already existing cells with same values will still be copied.
====
See Jonathan Hsieh's link:https://blog.cloudera.com/blog/2012/06/online-hbase-backups-with-copytable-2/[Online
HBase Backups with CopyTable] blog post for more on `CopyTable`.
@ -531,21 +562,22 @@ $ ./bin/hbase org.apache.hadoop.hbase.mapreduce.HashTable --help
Usage: HashTable [options] <tablename> <outputpath>
Options:
batchsize the target amount of bytes to hash in each batch
rows are added to the batch until this size is reached
(defaults to 8000 bytes)
numhashfiles the number of hash files to create
if set to fewer than number of regions then
the job will create this number of reducers
(defaults to 1/100 of regions -- at least 1)
startrow the start row
stoprow the stop row
starttime beginning of the time range (unixtime in millis)
without endtime means from starttime to forever
endtime end of the time range. Ignored if no starttime specified.
scanbatch scanner batch size to support intra row scans
versions number of cell versions to include
families comma-separated list of families to include
batchsize the target amount of bytes to hash in each batch
rows are added to the batch until this size is reached
(defaults to 8000 bytes)
numhashfiles the number of hash files to create
if set to fewer than number of regions then
the job will create this number of reducers
(defaults to 1/100 of regions -- at least 1)
startrow the start row
stoprow the stop row
starttime beginning of the time range (unixtime in millis)
without endtime means from starttime to forever
endtime end of the time range. Ignored if no starttime specified.
scanbatch scanner batch size to support intra row scans
versions number of cell versions to include
families comma-separated list of families to include
ignoreTimestamps if true, ignores cell timestamps
Args:
tablename Name of the table to hash
@ -584,6 +616,10 @@ Options:
(defaults to true)
doPuts if false, does not perform puts
(defaults to true)
ignoreTimestamps if true, ignores cells timestamps while comparing
cell values. Any missing cell on target then gets
added with current time as timestamp
(defaults to false)
Args:
sourcehashdir path to HashTable output dir for source table
@ -597,6 +633,13 @@ Examples:
$ bin/hbase org.apache.hadoop.hbase.mapreduce.SyncTable --dryrun=true --sourcezkcluster=zk1.example.com,zk2.example.com,zk3.example.com:2181:/hbase hdfs://nn:9000/hashes/tableA tableA tableA
----
Cell comparison takes ROW/FAMILY/QUALIFIER/TIMESTAMP/VALUE into account for equality. When syncing at the target, missing cells will be
added with original timestamp value from source. That may cause unexpected results after SyncTable completes, for example, if missing
cells on target have a delete marker with a timestamp T2 (say, a bulk delete performed by mistake), but source cells timestamps have an
older value T1, then those cells would still be unavailable at target because of the newer delete marker timestamp. Since cell timestamps
might not be relevant to all use cases, _ignoreTimestamps_ option adds the flexibility to avoid using cells timestamp in the comparison.
When using _ignoreTimestamps_ set to true, this option must be specified for both HashTable and SyncTable steps.
The *dryrun* option is useful when a read only, diff report is wanted, as it will produce only COUNTERS indicating the differences, but will not perform
any actual changes. It can be used as an alternative to VerifyReplication tool.
@ -606,6 +649,7 @@ Setting doDeletes to false modifies default behaviour to not delete target cells
Similarly, setting doPuts to false modifies default behaviour to not add missing cells on target. Setting both doDeletes
and doPuts to false would give same effect as setting dryrun to true.
.Additional info on doDeletes/doPuts
[NOTE]
====
@ -616,6 +660,16 @@ For major 1.x versions, minimum minor release including it is *1.4.10*.
For major 2.x versions, minimum minor release including it is *2.1.5*.
====
.Additional info on ignoreTimestamps
[NOTE]
====
"ignoreTimestamps" was only added by
link:https://issues.apache.org/jira/browse/HBASE-24302[HBASE-24302], so it may not be available on
all released versions.
For major 1.x versions, minimum minor release including it is *1.4.14*.
For major 2.x versions, minimum minor release including it is *2.2.5*.
====
.Set doDeletes to false on Two-Way Replication scenarios
[NOTE]
====
@ -633,8 +687,11 @@ which does not give any meaningful result.
.Remote Clusters on different Kerberos Realms
[NOTE]
====
Currently, SyncTable can't be ran for remote clusters on different Kerberos realms.
There's some work in progress to resolve this on link:https://jira.apache.org/jira/browse/HBASE-20586[HBASE-20586]
Often, remote clusters may be deployed on different Kerberos Realms.
link:https://jira.apache.org/jira/browse/HBASE-20586[HBASE-20586] added SyncTable support for
cross realm authentication, allowing a SyncTable process running on target cluster to connect to
source cluster and read both HashTable output files and the given HBase table when performing the
required comparisons.
====
[[export]]
@ -847,6 +904,13 @@ The output can optionally be mapped to another set of tables.
WALPlayer can also generate HFiles for later bulk importing, in that case only a single table and no mapping can be specified.
.WALPrettyPrinter/FSHLog Tool
[NOTE]
====
To read or verify single WAL files or _recovered.edits_ files, since they share the WAL format,
see <<_wal_tools>>.
====
Invoke via:
----
@ -947,15 +1011,85 @@ See link:https://issues.apache.org/jira/browse/HBASE-4391[HBASE-4391 Add ability
[[compaction.tool]]
=== Offline Compaction Tool
See the usage for the
link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/CompactionTool.html[CompactionTool].
Run it like:
*CompactionTool* provides a way of running compactions (either minor or major) as an independent
process from the RegionServer. It reuses same internal implementation classes executed by RegionServer
compaction feature. However, since this runs on a complete separate independent java process, it
releases RegionServers from the overhead involved in rewrite a set of hfiles, which can be critical
for latency sensitive use cases.
[source, bash]
Usage:
----
$ ./bin/hbase org.apache.hadoop.hbase.regionserver.CompactionTool
Usage: java org.apache.hadoop.hbase.regionserver.CompactionTool \
[-compactOnce] [-major] [-mapred] [-D<property=value>]* files...
Options:
mapred Use MapReduce to run compaction.
compactOnce Execute just one compaction step. (default: while needed)
major Trigger major compaction.
Note: -D properties will be applied to the conf used.
For example:
To stop delete of compacted file, pass -Dhbase.compactiontool.delete=false
To set tmp dir, pass -Dhbase.tmp.dir=ALTERNATE_DIR
Examples:
To compact the full 'TestTable' using MapReduce:
$ hbase org.apache.hadoop.hbase.regionserver.CompactionTool -mapred hdfs://hbase/data/default/TestTable
To compact column family 'x' of the table 'TestTable' region 'abc':
$ hbase org.apache.hadoop.hbase.regionserver.CompactionTool hdfs://hbase/data/default/TestTable/abc/x
----
As shown by usage options above, *CompactionTool* can run as a standalone client or a mapreduce job.
When running as mapreduce job, each family dir is handled as an input split, and is processed
by a separate map task.
The *compactionOnce* parameter controls how many compaction cycles will be performed until
*CompactionTool* program decides to finish its work. If omitted, it will assume it should keep
running compactions on each specified family as determined by the given compaction policy
configured. For more info on compaction policy, see <<compaction,compaction>>.
If a major compaction is desired, *major* flag can be specified. If omitted, *CompactionTool* will
assume minor compaction is wanted by default.
It also allows for configuration overrides with `-D` flag. In the usage section above, for example,
`-Dhbase.compactiontool.delete=false` option will instruct compaction engine to not delete original
files from temp folder.
Files targeted for compaction must be specified as parent hdfs dirs. It allows for multiple dirs
definition, as long as each for these dirs are either a *family*, a *region*, or a *table* dir. If a
table or region dir is passed, the program will recursively iterate through related sub-folders,
effectively running compaction for each family found below the table/region level.
Since these dirs are nested under *hbase* hdfs directory tree, *CompactionTool* requires hbase super
user permissions in order to have access to required hfiles.
.Running in MapReduce mode
[NOTE]
====
MapReduce mode offers the ability to process each family dir in parallel, as a separate map task.
Generally, it would make sense to run in this mode when specifying one or more table dirs as targets
for compactions. The caveat, though, is that if number of families to be compacted become too large,
the related mapreduce job may have indirect impacts on *RegionServers* performance .
Since *NodeManagers* are normally co-located with RegionServers, such large jobs could
compete for IO/Bandwidth resources with the *RegionServers*.
====
.MajorCompaction completely disabled on RegionServers due performance impacts
[NOTE]
====
*Major compactions* can be a costly operation (see <<compaction,compaction>>), and can indeed
impact performance on RegionServers, leading operators to completely disable it for critical
low latency application. *CompactionTool* could be used as an alternative in such scenarios,
although, additional custom application logic would need to be implemented, such as deciding
scheduling and selection of tables/regions/families target for a given compaction run.
====
For additional details about CompactionTool, see also
link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/CompactionTool.html[CompactionTool].
=== `hbase clean`
The `hbase clean` command cleans HBase data from ZooKeeper, HDFS, or both.
@ -1327,7 +1461,7 @@ But usually disks do the "John Wayne" -- i.e.
take a while to go down spewing errors in _dmesg_ -- or for some reason, run much slower than their companions.
In this case you want to decommission the disk.
You have two options.
You can link:https://wiki.apache.org/hadoop/FAQ#I_want_to_make_a_large_cluster_smaller_by_taking_out_a_bunch_of_nodes_simultaneously._How_can_this_be_done.3F[decommission
You can link:https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDataNodeAdminGuide.html[decommission
the datanode] or, less disruptive in that only the bad disks data will be rereplicated, can stop the datanode, unmount the bad volume (You can't umount a volume while the datanode is using it), and then restart the datanode (presuming you have set dfs.datanode.failed.volumes.tolerated > 0). The regionserver will throw some errors in its logs as it recalibrates where to get its data from -- it will likely roll its WAL log too -- but in general but for some latency spikes, it should keep on chugging.
.Short Circuit Reads
@ -1400,15 +1534,6 @@ Monitor the output of the _/tmp/log.txt_ file to follow the progress of the scri
Use the following guidelines if you want to create your own rolling restart script.
. Extract the new release, verify its configuration, and synchronize it to all nodes of your cluster using `rsync`, `scp`, or another secure synchronization mechanism.
. Use the hbck utility to ensure that the cluster is consistent.
+
----
$ ./bin/hbck
----
+
Perform repairs if required.
See <<hbck,hbck>> for details.
. Restart the master first.
You may need to modify these commands if your new HBase directory is different from the old one, such as for an upgrade.
@ -1440,7 +1565,6 @@ $ for i in `cat conf/regionservers|sort`; do ./bin/graceful_stop.sh --restart --
----
. Restart the Master again, to clear out the dead servers list and re-enable the load balancer.
. Run the `hbck` utility again, to be sure the cluster is consistent.
[[adding.new.node]]
=== Adding a New Node
@ -1996,24 +2120,46 @@ include::slow_log_responses_from_systable.adoc[]
=== Block Cache Monitoring
Starting with HBase 0.98, the HBase Web UI includes the ability to monitor and report on the performance of the block cache.
To view the block cache reports, click .
To view the block cache reports, see the Block Cache section of the region server UI.
Following are a few examples of the reporting capabilities.
.Basic Info
.Basic Info shows the cache implementation.
image::bc_basic.png[]
.Config
.Config shows all cache configuration options.
image::bc_config.png[]
.Stats
.Stats shows statistics about the performance of the cache.
image::bc_stats.png[]
.L1 and L2
.L1 and L2 show information about the L1 and L2 caches.
image::bc_l1.png[]
This is not an exhaustive list of all the screens and reports available.
Have a look in the Web UI.
=== Snapshot Space Usage Monitoring
Starting with HBase 0.95, Snapshot usage information on individual snapshots was shown in the HBase Master Web UI. This was further enhanced starting with HBase 1.3 to show the total Storefile size of the Snapshot Set. The following metrics are shown in the Master Web UI with HBase 1.3 and later.
* Shared Storefile Size is the Storefile size shared between snapshots and active tables.
* Mob Storefile Size is the Mob Storefile size shared between snapshots and active tables.
* Archived Storefile Size is the Storefile size in Archive.
The format of Archived Storefile Size is NNN(MMM). NNN is the total Storefile size in Archive, MMM is the total Storefile size in Archive that is specific to the snapshot (not shared with other snapshots and tables).
.Master Snapshot Overview
image::master-snapshot.png[]
.Snapshot Storefile Stats Example 1
image::1-snapshot.png[]
.Snapshot Storefile Stats Example 2
image::2-snapshots.png[]
.Empty Snapshot Storfile Stats Example
image::empty-snapshots.png[]
== Cluster Replication
NOTE: This information was previously available at
@ -2030,6 +2176,9 @@ Some use cases for cluster replication include:
NOTE: Replication is enabled at the granularity of the column family.
Before enabling replication for a column family, create the table and all column families to be replicated, on the destination cluster.
NOTE: Replication is asynchronous as we send WAL to another cluster in background, which means that when you want to do recovery through replication, you could loss some data. To address this problem, we have introduced a new feature called synchronous replication. As the mechanism is a bit different so we use a separated section to describe it. Please see
<<Synchronous Replication,Synchronous Replication>>.
=== Replication Overview
Cluster replication uses a source-push methodology.
@ -2852,14 +3001,17 @@ Since the cluster is up, there is a risk that edits could be missed in the copy
The <<export,export>> approach dumps the content of a table to HDFS on the same cluster.
To restore the data, the <<import,import>> utility would be used.
Since the cluster is up, there is a risk that edits could be missed in the export process.
Since the cluster is up, there is a risk that edits could be missed in the export process. If you want to know more about HBase back-up and restore see the page on link:http://hbase.apache.org/book.html#backuprestore[Backup and Restore].
[[ops.snapshots]]
== HBase Snapshots
HBase Snapshots allow you to take a snapshot of a table without too much impact on Region Servers.
Snapshot, Clone and restore operations don't involve data copying.
Also, Exporting the snapshot to another cluster doesn't have impact on the Region Servers.
HBase Snapshots allow you to take a copy of a table (both contents and metadata)with a very small performance impact. A Snapshot is an immutable
collection of table metadata and a list of HFiles that comprised the table at the time the Snapshot was taken. A "clone"
of a snapshot creates a new table from that snapshot, and a "restore" of a snapshot returns the contents of a table to
what it was when the snapshot was created. The "clone" and "restore" operations do not require any data to be copied,
as the underlying HFiles (the files which contain the data for an HBase table) are not modified with either action.
Simiarly, exporting a snapshot to another cluster has little impact on RegionServers of the local cluster.
Prior to version 0.94.6, the only way to backup or to clone a table is to use CopyTable/ExportTable, or to copy all the hfiles in HDFS after disabling the table.
The disadvantages of these methods are that you can degrade region server performance (Copy/Export Table) or you need to disable the table, that means no reads or writes; and this is usually unacceptable.
@ -3193,8 +3345,6 @@ HDFS replication factor only affects your disk usage and is invisible to most HB
You can view the current number of regions for a given table using the HMaster UI.
In the [label]#Tables# section, the number of online regions for each table is listed in the [label]#Online Regions# column.
This total only includes the in-memory state and does not include disabled or offline regions.
If you do not want to use the HMaster UI, you can determine the number of regions by counting the number of subdirectories of the /hbase/<table>/ subdirectories in HDFS, or by running the `bin/hbase hbck` command.
Each of these methods may return a slightly different number, depending on the status of each region.
[[ops.capacity.regions.count]]
==== Number of regions per RS - upper bound
@ -3481,8 +3631,8 @@ If it appears stuck, restart the Master process.
=== Remove RegionServer Grouping
Removing RegionServer Grouping feature from a cluster on which it was enabled involves
more steps in addition to removing the relevant properties from `hbase-site.xml`. This is
to clean the RegionServer grouping related meta data so that if the feature is re-enabled
more steps in addition to removing the relevant properties from `hbase-site.xml`. This is
to clean the RegionServer grouping related meta data so that if the feature is re-enabled
in the future, the old meta data will not affect the functioning of the cluster.
- Move all tables in non-default rsgroups to `default` regionserver group
@ -3491,7 +3641,7 @@ in the future, the old meta data will not affect the functioning of the cluster.
#Reassigning table t1 from non default group - hbase shell
hbase(main):005:0> move_tables_rsgroup 'default',['t1']
----
- Move all regionservers in non-default rsgroups to `default` regionserver group
- Move all regionservers in non-default rsgroups to `default` regionserver group
[source, bash]
----
#Reassigning all the servers in the non-default rsgroup to default - hbase shell
@ -3575,21 +3725,21 @@ To check normalizer status and enable/disable normalizer
[source,bash]
----
hbase(main):001:0> normalizer_enabled
true
true
0 row(s) in 0.4870 seconds
hbase(main):002:0> normalizer_switch false
true
true
0 row(s) in 0.0640 seconds
hbase(main):003:0> normalizer_enabled
false
false
0 row(s) in 0.0120 seconds
hbase(main):004:0> normalizer_switch true
false
0 row(s) in 0.0200 seconds
hbase(main):005:0> normalizer_enabled
true
0 row(s) in 0.0090 seconds
@ -3608,19 +3758,19 @@ merge action being taken as a result of the normalization plan computed by Simpl
Consider an user table with some pre-split regions having 3 equally large regions
(about 100K rows) and 1 relatively small region (about 25K rows). Following is the
snippet from an hbase meta table scan showing each of the pre-split regions for
snippet from an hbase meta table scan showing each of the pre-split regions for
the user table.
----
table_p8ddpd6q5z,,1469494305548.68b9892220865cb6048 column=info:regioninfo, timestamp=1469494306375, value={ENCODED => 68b9892220865cb604809c950d1adf48, NAME => 'table_p8ddpd6q5z,,1469494305548.68b989222 09c950d1adf48. 0865cb604809c950d1adf48.', STARTKEY => '', ENDKEY => '1'}
....
table_p8ddpd6q5z,1,1469494317178.867b77333bdc75a028 column=info:regioninfo, timestamp=1469494317848, value={ENCODED => 867b77333bdc75a028bb4c5e4b235f48, NAME => 'table_p8ddpd6q5z,1,1469494317178.867b7733 bb4c5e4b235f48. 3bdc75a028bb4c5e4b235f48.', STARTKEY => '1', ENDKEY => '3'}
....
table_p8ddpd6q5z,3,1469494328323.98f019a753425e7977 column=info:regioninfo, timestamp=1469494328486, value={ENCODED => 98f019a753425e7977ab8636e32deeeb, NAME => 'table_p8ddpd6q5z,3,1469494328323.98f019a7 ab8636e32deeeb. 53425e7977ab8636e32deeeb.', STARTKEY => '3', ENDKEY => '7'}
....
table_p8ddpd6q5z,7,1469494339662.94c64e748979ecbb16 column=info:regioninfo, timestamp=1469494339859, value={ENCODED => 94c64e748979ecbb166f6cc6550e25c6, NAME => 'table_p8ddpd6q5z,7,1469494339662.94c64e74 6f6cc6550e25c6. 8979ecbb166f6cc6550e25c6.', STARTKEY => '7', ENDKEY => '8'}
....
table_p8ddpd6q5z,8,1469494339662.6d2b3f5fd1595ab8e7 column=info:regioninfo, timestamp=1469494339859, value={ENCODED => 6d2b3f5fd1595ab8e7c031876057b1ee, NAME => 'table_p8ddpd6q5z,8,1469494339662.6d2b3f5f c031876057b1ee. d1595ab8e7c031876057b1ee.', STARTKEY => '8', ENDKEY => ''}
table_p8ddpd6q5z,,1469494305548.68b9892220865cb6048 column=info:regioninfo, timestamp=1469494306375, value={ENCODED => 68b9892220865cb604809c950d1adf48, NAME => 'table_p8ddpd6q5z,,1469494305548.68b989222 09c950d1adf48. 0865cb604809c950d1adf48.', STARTKEY => '', ENDKEY => '1'}
....
table_p8ddpd6q5z,1,1469494317178.867b77333bdc75a028 column=info:regioninfo, timestamp=1469494317848, value={ENCODED => 867b77333bdc75a028bb4c5e4b235f48, NAME => 'table_p8ddpd6q5z,1,1469494317178.867b7733 bb4c5e4b235f48. 3bdc75a028bb4c5e4b235f48.', STARTKEY => '1', ENDKEY => '3'}
....
table_p8ddpd6q5z,3,1469494328323.98f019a753425e7977 column=info:regioninfo, timestamp=1469494328486, value={ENCODED => 98f019a753425e7977ab8636e32deeeb, NAME => 'table_p8ddpd6q5z,3,1469494328323.98f019a7 ab8636e32deeeb. 53425e7977ab8636e32deeeb.', STARTKEY => '3', ENDKEY => '7'}
....
table_p8ddpd6q5z,7,1469494339662.94c64e748979ecbb16 column=info:regioninfo, timestamp=1469494339859, value={ENCODED => 94c64e748979ecbb166f6cc6550e25c6, NAME => 'table_p8ddpd6q5z,7,1469494339662.94c64e74 6f6cc6550e25c6. 8979ecbb166f6cc6550e25c6.', STARTKEY => '7', ENDKEY => '8'}
....
table_p8ddpd6q5z,8,1469494339662.6d2b3f5fd1595ab8e7 column=info:regioninfo, timestamp=1469494339859, value={ENCODED => 6d2b3f5fd1595ab8e7c031876057b1ee, NAME => 'table_p8ddpd6q5z,8,1469494339662.6d2b3f5f c031876057b1ee. d1595ab8e7c031876057b1ee.', STARTKEY => '8', ENDKEY => ''}
----
Invoking the normalizer using normalize int the HBase shell, the below log snippet
from HMaster log shows the normalization plan computed as per the logic defined for
@ -3646,15 +3796,15 @@ and end key as 1, with another region having start key as 1 and end
Now, that these regions have been merged we see a single new region with start key
as and end key as 3
----
table_p8ddpd6q5z,,1469516907210.e06c9b83c4a252b130e column=info:mergeA, timestamp=1469516907431,
value=PBUF\x08\xA5\xD9\x9E\xAF\xE2*\x12\x1B\x0A\x07default\x12\x10table_p8ddpd6q5z\x1A\x00"\x011(\x000\x00 ea74d246741ba. 8\x00
table_p8ddpd6q5z,,1469516907210.e06c9b83c4a252b130e column=info:mergeA, timestamp=1469516907431,
value=PBUF\x08\xA5\xD9\x9E\xAF\xE2*\x12\x1B\x0A\x07default\x12\x10table_p8ddpd6q5z\x1A\x00"\x011(\x000\x00 ea74d246741ba. 8\x00
table_p8ddpd6q5z,,1469516907210.e06c9b83c4a252b130e column=info:mergeB, timestamp=1469516907431,
value=PBUF\x08\xB5\xBA\x9F\xAF\xE2*\x12\x1B\x0A\x07default\x12\x10table_p8ddpd6q5z\x1A\x011"\x013(\x000\x0 ea74d246741ba. 08\x00
value=PBUF\x08\xB5\xBA\x9F\xAF\xE2*\x12\x1B\x0A\x07default\x12\x10table_p8ddpd6q5z\x1A\x011"\x013(\x000\x0 ea74d246741ba. 08\x00
table_p8ddpd6q5z,,1469516907210.e06c9b83c4a252b130e column=info:regioninfo, timestamp=1469516907431, value={ENCODED => e06c9b83c4a252b130eea74d246741ba, NAME => 'table_p8ddpd6q5z,,1469516907210.e06c9b83c ea74d246741ba. 4a252b130eea74d246741ba.', STARTKEY => '', ENDKEY => '3'}
....
table_p8ddpd6q5z,3,1469514778736.bf024670a847c0adff column=info:regioninfo, timestamp=1469514779417, value={ENCODED => bf024670a847c0adffb74b2e13408b32, NAME => 'table_p8ddpd6q5z,3,1469514778736.bf024670 b74b2e13408b32. a847c0adffb74b2e13408b32.' STARTKEY => '3', ENDKEY => '7'}
....
table_p8ddpd6q5z,7,1469514790152.7c5a67bc755e649db2 column=info:regioninfo, timestamp=1469514790312, value={ENCODED => 7c5a67bc755e649db22f49af6270f1e1, NAME => 'table_p8ddpd6q5z,7,1469514790152.7c5a67bc 2f49af6270f1e1. 755e649db22f49af6270f1e1.', STARTKEY => '7', ENDKEY => '8'}
....
table_p8ddpd6q5z,3,1469514778736.bf024670a847c0adff column=info:regioninfo, timestamp=1469514779417, value={ENCODED => bf024670a847c0adffb74b2e13408b32, NAME => 'table_p8ddpd6q5z,3,1469514778736.bf024670 b74b2e13408b32. a847c0adffb74b2e13408b32.' STARTKEY => '3', ENDKEY => '7'}
....
table_p8ddpd6q5z,7,1469514790152.7c5a67bc755e649db2 column=info:regioninfo, timestamp=1469514790312, value={ENCODED => 7c5a67bc755e649db22f49af6270f1e1, NAME => 'table_p8ddpd6q5z,7,1469514790152.7c5a67bc 2f49af6270f1e1. 755e649db22f49af6270f1e1.', STARTKEY => '7', ENDKEY => '8'}
....
table_p8ddpd6q5z,8,1469514790152.58e7503cda69f98f47 column=info:regioninfo, timestamp=1469514790312, value={ENCODED => 58e7503cda69f98f4755178e74288c3a, NAME => 'table_p8ddpd6q5z,8,1469514790152.58e7503c 55178e74288c3a. da69f98f4755178e74288c3a.', STARTKEY => '8', ENDKEY => ''}
----
@ -3682,6 +3832,7 @@ server=hbase-test-rc-5.openstacklocal,16020,1469419333913}
----
[[auto_reopen_regions]]
== Auto Region Reopen

View File

@ -257,12 +257,12 @@ The following examples illustrate some of the possibilities.
Note that you always have at least one write queue, no matter what setting you use.
* The default value of `0` does not split the queue.
* A value of `.3` uses 30% of the queues for reading and 60% for writing.
* A value of `.3` uses 30% of the queues for reading and 70% for writing.
Given a value of `10` for `hbase.ipc.server.num.callqueue`, 3 queues would be used for reads and 7 for writes.
* A value of `.5` uses the same number of read queues and write queues.
Given a value of `10` for `hbase.ipc.server.num.callqueue`, 5 queues would be used for reads and 5 for writes.
* A value of `.6` uses 60% of the queues for reading and 30% for reading.
Given a value of `10` for `hbase.ipc.server.num.callqueue`, 7 queues would be used for reads and 3 for writes.
* A value of `.6` uses 60% of the queues for reading and 40% for reading.
Given a value of `10` for `hbase.ipc.server.num.callqueue`, 6 queues would be used for reads and 4 for writes.
* A value of `1.0` uses one queue to process write requests, and all other queues process read requests.
A value higher than `1.0` has the same effect as a value of `1.0`.
Given a value of `10` for `hbase.ipc.server.num.callqueue`, 9 queues would be used for reads and 1 for writes.
@ -273,11 +273,11 @@ More queues are used for Gets if the value is below `.5` and more are used for s
No matter what setting you use, at least one read queue is used for Get operations.
* A value of `0` does not split the read queue.
* A value of `.3` uses 60% of the read queues for Gets and 30% for Scans.
* A value of `.3` uses 70% of the read queues for Gets and 30% for Scans.
Given a value of `20` for `hbase.ipc.server.num.callqueue` and a value of `.5` for `hbase.ipc.server.callqueue.read.ratio`, 10 queues would be used for reads, out of those 10, 7 would be used for Gets and 3 for Scans.
* A value of `.5` uses half the read queues for Gets and half for Scans.
Given a value of `20` for `hbase.ipc.server.num.callqueue` and a value of `.5` for `hbase.ipc.server.callqueue.read.ratio`, 10 queues would be used for reads, out of those 10, 5 would be used for Gets and 5 for Scans.
* A value of `.6` uses 30% of the read queues for Gets and 60% for Scans.
* A value of `.7` uses 30% of the read queues for Gets and 70% for Scans.
Given a value of `20` for `hbase.ipc.server.num.callqueue` and a value of `.5` for `hbase.ipc.server.callqueue.read.ratio`, 10 queues would be used for reads, out of those 10, 3 would be used for Gets and 7 for Scans.
* A value of `1.0` uses all but one of the read queues for Scans.
Given a value of `20` for `hbase.ipc.server.num.callqueue` and a value of`.5` for `hbase.ipc.server.callqueue.read.ratio`, 10 queues would be used for reads, out of those 10, 1 would be used for Gets and 9 for Scans.

View File

@ -68,7 +68,7 @@ Yours, the HBase Community.
Please use link:https://issues.apache.org/jira/browse/hbase[JIRA] to report non-security-related bugs.
To protect existing HBase installations from new vulnerabilities, please *do not* use JIRA to report security-related bugs. Instead, send your report to the mailing list private@apache.org, which allows anyone to send messages, but restricts who can read them. Someone on that list will contact you to follow up on your report.
To protect existing HBase installations from new vulnerabilities, please *do not* use JIRA to report security-related bugs. Instead, send your report to the mailing list private@hbase.apache.org, which allows anyone to send messages, but restricts who can read them. Someone on that list will contact you to follow up on your report.
[[hbase_supported_tested_definitions]]
.Support and Testing Expectations

View File

@ -34,6 +34,9 @@ HBASE-21926 introduced a new servlet that supports integrated profiling via asyn
== Prerequisites
Go to https://github.com/jvm-profiling-tools/async-profiler, download a release appropriate for your platform, and install on every cluster host.
If 4.6 or later linux, be sure to set proc variables as per 'Basic Usage' section in the
<a href="https://github.com/jvm-profiling-tools/async-profiler">Async Profiler Home Page</a>
(Not doing this will draw you diagrams with no content).
Set `ASYNC_PROFILER_HOME` in the environment (put it in hbase-env.sh) to the root directory of the async-profiler install location, or pass it on the HBase daemon's command line as a system property as `-Dasync.profiler.home=/path/to/async-profiler`.

View File

@ -148,3 +148,75 @@ consider extending it also in
Going forward, we will provide a new module of common types for use
by CPEPs that will have the same guarantees against change as does our
public API. TODO.
=== protobuf changes for hbase-3.0.0 (HBASE-23797)
Since hadoop(start from 3.3.x) also shades protobuf and bumps the version to
3.x, there is no reason for us to stay on protobuf 2.5.0 any more.
In HBase 3.0.0, the hbase-protocol module has been purged, the CPEP
implementation should use the protos in hbase-protocol-shaded module, and also
make use of the shaded protobuf in hbase-thirdparty. In general, we will keep
the protobuf version compatible for a whole major release, unless there are
critical problems, for example, a critical CVE on protobuf.
Add this dependency to your pom:
[source,xml]
----
<dependency>
<groupId>org.apache.hbase.thirdparty</groupId>
<artifactId>hbase-shaded-protobuf</artifactId>
<!-- use the version that your target hbase cluster uses -->
<version>${hbase-thirdparty.version}</version>
<scope>provided</scope>
</dependency>
----
And typically you also need to add this plugin to your pom to make your
generated protobuf code also use the shaded and relocated protobuf version
in hbase-thirdparty.
[source,xml]
----
<plugin>
<groupId>com.google.code.maven-replacer-plugin</groupId>
<artifactId>replacer</artifactId>
<version>1.5.3</version>
<executions>
<execution>
<phase>process-sources</phase>
<goals>
<goal>replace</goal>
</goals>
</execution>
</executions>
<configuration>
<basedir>${basedir}/target/generated-sources/</basedir>
<includes>
<include>**/*.java</include>
</includes>
<!-- Ignore errors when missing files, because it means this build
was run with -Dprotoc.skip and there is no -Dreplacer.skip -->
<ignoreErrors>true</ignoreErrors>
<replacements>
<replacement>
<token>([^\.])com.google.protobuf</token>
<value>$1org.apache.hbase.thirdparty.com.google.protobuf</value>
</replacement>
<replacement>
<token>(public)(\W+static)?(\W+final)?(\W+class)</token>
<value>@javax.annotation.Generated("proto") $1$2$3$4</value>
</replacement>
<!-- replacer doesn't support anchoring or negative lookbehind -->
<replacement>
<token>(@javax.annotation.Generated\("proto"\) ){2}</token>
<value>$1</value>
</replacement>
</replacements>
</configuration>
</plugin>
----
In hbase-examples module, we have some examples under the
`org.apache.hadoop.hbase.coprocessor.example` package. You can see
`BulkDeleteEndpoint` and `BulkDelete.proto` for more details, and you can also
check the `pom.xml` of hbase-examples module to see how to make use of the above
plugin.

View File

@ -1143,7 +1143,11 @@ Disable Nagles algorithm. Delayed ACKs can add up to ~200ms to RPC round trip
Detect regionserver failure as fast as reasonable. Set the following parameters:
* In `hbase-site.xml`, set `zookeeper.session.timeout` to 30 seconds or less to bound failure detection (20-30 seconds is a good start).
- Notice: the `sessionTimeout` of zookeeper is limited between 2 times and 20 times the `tickTime`(the basic time unit in milliseconds used by ZooKeeper.the default value is 2000ms.It is used to do heartbeats and the minimum session timeout will be twice the tickTime).
- Note: Zookeeper clients negotiate a session timeout with the server during client init. Server enforces this timeout to be in the
range [`minSessionTimeout`, `maxSessionTimeout`] and both these timeouts (measured in milliseconds) are configurable in Zookeeper service configuration.
If not configured, these default to 2 * `tickTime` and 20 * `tickTime` respectively (`tickTime` is the basic time unit used by ZooKeeper,
as measured in milliseconds. It is used to regulate heartbeats, timeouts etc.). Refer to Zookeeper documentation for additional details.
* Detect and avoid unhealthy or failed HDFS DataNodes: in `hdfs-site.xml` and `hbase-site.xml`, set the following parameters:
- `dfs.namenode.avoid.read.stale.datanode = true`
- `dfs.namenode.avoid.write.stale.datanode = true`
@ -1160,7 +1164,7 @@ the regionserver/dfsclient side.
* In `hbase-site.xml`, set the following parameters:
- `dfs.client.read.shortcircuit = true`
- `dfs.client.read.shortcircuit.skip.checksum = true` so we don't double checksum (HBase does its own checksumming to save on i/os. See <<hbase.regionserver.checksum.verify.performance>> for more on this.
- `dfs.client.read.shortcircuit.skip.checksum = true` so we don't double checksum (HBase does its own checksumming to save on i/os. See <<hbase.regionserver.checksum.verify.performance>> for more on this.
- `dfs.domain.socket.path` to match what was set for the datanodes.
- `dfs.client.read.shortcircuit.buffer.size = 131072` Important to avoid OOME -- hbase has a default it uses if unset, see `hbase.dfs.client.read.shortcircuit.buffer.size`; its default is 131072.
* Ensure data locality. In `hbase-site.xml`, set `hbase.hstore.min.locality.to.skip.major.compact = 0.7` (Meaning that 0.7 \<= n \<= 1)

View File

@ -30,7 +30,7 @@
[IMPORTANT]
.Reporting Security Bugs
====
NOTE: To protect existing HBase installations from exploitation, please *do not* use JIRA to report security-related bugs. Instead, send your report to the mailing list private@apache.org, which allows anyone to send messages, but restricts who can read them. Someone on that list will contact you to follow up on your report.
NOTE: To protect existing HBase installations from exploitation, please *do not* use JIRA to report security-related bugs. Instead, send your report to the mailing list private@hbase.apache.org, which allows anyone to send messages, but restricts who can read them. Someone on that list will contact you to follow up on your report.
HBase adheres to the Apache Software Foundation's policy on reported vulnerabilities, available at http://apache.org/security/.
@ -1811,7 +1811,7 @@ All options have been discussed separately in the sections above.
<!-- HBase Superuser -->
<property>
<name>hbase.superuser</name>
<value>hbase, admin</value>
<value>hbase,admin</value>
</property>
<!-- Coprocessors for ACLs and Visibility Tags -->
<property>
@ -1831,8 +1831,7 @@ All options have been discussed separately in the sections above.
</property>
<property>
<name>hbase.coprocessor.regionserver.classes</name>
<value>org.apache.hadoop/hbase.security.access.AccessController,
org.apache.hadoop.hbase.security.access.VisibilityController</value>
<value>org.apache.hadoop.hbase.security.access.AccessController</value>
</property>
<!-- Executable ACL for Coprocessor Endpoints -->
<property>

View File

@ -0,0 +1,152 @@
////
/**
*
* 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.
*/
////
[[snapshot_scanner]]
== Scan over snapshot
:doctype: book
:numbered:
:toc: left
:icons: font
:experimental:
:toc: left
:source-language: java
In HBase, a scan of a table costs server-side HBase resources reading, formating, and returning data back to the client.
Luckily, HBase provides a TableSnapshotScanner and TableSnapshotInputFormat (introduced by link:https://issues.apache.org/jira/browse/HBASE-8369[HBASE-8369]),
which can scan HBase-written HFiles directly in the HDFS filesystem completely by-passing hbase. This access mode
performs better than going via HBase and can be used with an offline HBase with in-place or exported
snapshot HFiles.
To read HFiles directly, the user must have sufficient permissions to access snapshots or in-place hbase HFiles.
=== TableSnapshotScanner
TableSnapshotScanner provides a means for running a single client-side scan over snapshot files.
When using TableSnapshotScanner, we must specify a temporary directory to copy the snapshot files into.
The client user should have write permissions to this directory, and the dir should not be a subdirectory of
the hbase.rootdir. The scanner deletes the contents of the directory once the scanner is closed.
.Use TableSnapshotScanner
====
[source,java]
----
Path restoreDir = new Path("XX"); // restore dir should not be a subdirectory of hbase.rootdir
Scan scan = new Scan();
try (TableSnapshotScanner scanner = new TableSnapshotScanner(conf, restoreDir, snapshotName, scan)) {
Result result = scanner.next();
while (result != null) {
...
result = scanner.next();
}
}
----
====
=== TableSnapshotInputFormat
TableSnapshotInputFormat provides a way to scan over snapshot HFiles in a MapReduce job.
.Use TableSnapshotInputFormat
====
[source,java]
----
Job job = new Job(conf);
Path restoreDir = new Path("XX"); // restore dir should not be a subdirectory of hbase.rootdir
Scan scan = new Scan();
TableMapReduceUtil.initTableSnapshotMapperJob(snapshotName, scan, MyTableMapper.class, MyMapKeyOutput.class, MyMapOutputValueWritable.class, job, true, restoreDir);
----
====
=== Permission to access snapshot and data files
Generally, only the HBase owner or the HDFS admin have the permission to access HFiles.
link:https://issues.apache.org/jira/browse/HBASE-18659[HBASE-18659] uses HDFS ACLs to make HBase granted user have permission to access snapshot files.
==== link:https://hadoop.apache.org/docs/r2.7.1/hadoop-project-dist/hadoop-hdfs/HdfsPermissionsGuide.html#ACLs_Access_Control_Lists[HDFS ACLs]
HDFS ACLs supports an "access ACL", which defines the rules to enforce during permission checks, and a "default ACL",
which defines the ACL entries that new child files or sub-directories receive automatically during creation.
Via HDFS ACLs, HBase syncs granted users with read permission to HFiles.
==== Basic idea
The HBase files are organized in the following ways:
* {hbase-rootdir}/.tmp/data/{namespace}/{table}
* {hbase-rootdir}/data/{namespace}/{table}
* {hbase-rootdir}/archive/data/{namespace}/{table}
* {hbase-rootdir}/.hbase-snapshot/{snapshotName}
So the basic idea is to add or remove HDFS ACLs to files of the global/namespace/table directory
when grant or revoke permission to global/namespace/table.
See the design doc in link:https://issues.apache.org/jira/browse/HBASE-18659[HBASE-18659] for more details.
==== Configuration to use this feature
* Firstly, make sure that HDFS ACLs are enabled and umask is set to 027
----
dfs.namenode.acls.enabled = true
fs.permissions.umask-mode = 027
----
* Add master coprocessor, please make sure the SnapshotScannerHDFSAclController is configured after AccessController
----
hbase.coprocessor.master.classes = "org.apache.hadoop.hbase.security.access.AccessController
,org.apache.hadoop.hbase.security.access.SnapshotScannerHDFSAclController"
----
* Enable this feature
----
hbase.acl.sync.to.hdfs.enable=true
----
* Modify table scheme to enable this feature for a specified table, this config is
false by default for every table, this means the HBase granted ACLs will not be synced to HDFS
----
alter 't1', CONFIGURATION => {'hbase.acl.sync.to.hdfs.enable' => 'true'}
----
==== Limitation
There are some limitations for this feature:
=====
If we enable this feature, some master operations such as grant, revoke, snapshot...
(See the design doc for more details) will be slower as we need to sync HDFS ACLs to related hfiles.
=====
=====
HDFS has a config which limits the max ACL entries num for one directory or file:
----
dfs.namenode.acls.max.entries = 32(default value)
----
The 32 entries include four fixed users for each directory or file: owner, group, other, and mask.
For a directory, the four users contain 8 ACL entries(access and default) and for a file, the four
users contain 4 ACL entries(access). This means there are 24 ACL entries left for named users or groups.
Based on this limitation, we can only sync up to 12 HBase granted users' ACLs. This means, if a table
enables this feature, then the total users with table, namespace of this table, global READ permission
should not be greater than 12.
=====
=====
There are some cases that this coprocessor has not handled or could not handle, so the user HDFS ACLs
are not synced normally. It will not make a reference link to another hfile of other tables.
=====

View File

@ -0,0 +1,699 @@
////
/**
*
* 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.
*/
////
[[spark]]
= HBase and Spark
:doctype: book
:numbered:
:toc: left
:icons: font
:experimental:
link:https://spark.apache.org/[Apache Spark] is a software framework that is used
to process data in memory in a distributed manner, and is replacing MapReduce in
many use cases.
Spark itself is out of scope of this document, please refer to the Spark site for
more information on the Spark project and subprojects. This document will focus
on 4 main interaction points between Spark and HBase. Those interaction points are:
Basic Spark::
The ability to have an HBase Connection at any point in your Spark DAG.
Spark Streaming::
The ability to have an HBase Connection at any point in your Spark Streaming
application.
Spark Bulk Load::
The ability to write directly to HBase HFiles for bulk insertion into HBase
SparkSQL/DataFrames::
The ability to write SparkSQL that draws on tables that are represented in HBase.
The following sections will walk through examples of all these interaction points.
== Basic Spark
This section discusses Spark HBase integration at the lowest and simplest levels.
All the other interaction points are built upon the concepts that will be described
here.
At the root of all Spark and HBase integration is the HBaseContext. The HBaseContext
takes in HBase configurations and pushes them to the Spark executors. This allows
us to have an HBase Connection per Spark Executor in a static location.
For reference, Spark Executors can be on the same nodes as the Region Servers or
on different nodes, there is no dependence on co-location. Think of every Spark
Executor as a multi-threaded client application. This allows any Spark Tasks
running on the executors to access the shared Connection object.
.HBaseContext Usage Example
====
This example shows how HBaseContext can be used to do a `foreachPartition` on a RDD
in Scala:
[source, scala]
----
val sc = new SparkContext("local", "test")
val config = new HBaseConfiguration()
...
val hbaseContext = new HBaseContext(sc, config)
rdd.hbaseForeachPartition(hbaseContext, (it, conn) => {
val bufferedMutator = conn.getBufferedMutator(TableName.valueOf("t1"))
it.foreach((putRecord) => {
. val put = new Put(putRecord._1)
. putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
. bufferedMutator.mutate(put)
})
bufferedMutator.flush()
bufferedMutator.close()
})
----
Here is the same example implemented in Java:
[source, java]
----
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
try {
List<byte[]> list = new ArrayList<>();
list.add(Bytes.toBytes("1"));
...
list.add(Bytes.toBytes("5"));
JavaRDD<byte[]> rdd = jsc.parallelize(list);
Configuration conf = HBaseConfiguration.create();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.foreachPartition(rdd,
new VoidFunction<Tuple2<Iterator<byte[]>, Connection>>() {
public void call(Tuple2<Iterator<byte[]>, Connection> t)
throws Exception {
Table table = t._2().getTable(TableName.valueOf(tableName));
BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName));
while (t._1().hasNext()) {
byte[] b = t._1().next();
Result r = table.get(new Get(b));
if (r.getExists()) {
mutator.mutate(new Put(b));
}
}
mutator.flush();
mutator.close();
table.close();
}
});
} finally {
jsc.stop();
}
----
====
All functionality between Spark and HBase will be supported both in Scala and in
Java, with the exception of SparkSQL which will support any language that is
supported by Spark. For the remaining of this documentation we will focus on
Scala examples.
The examples above illustrate how to do a foreachPartition with a connection. A
number of other Spark base functions are supported out of the box:
// tag::spark_base_functions[]
`bulkPut`:: For massively parallel sending of puts to HBase
`bulkDelete`:: For massively parallel sending of deletes to HBase
`bulkGet`:: For massively parallel sending of gets to HBase to create a new RDD
`mapPartition`:: To do a Spark Map function with a Connection object to allow full
access to HBase
`hbaseRDD`:: To simplify a distributed scan to create a RDD
// end::spark_base_functions[]
For examples of all these functionalities, see the
link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
in the link:https://github.com/apache/hbase-connectors[hbase-connectors] repository
(the hbase-spark connectors live outside hbase core in a related,
Apache HBase project maintained, associated repo).
== Spark Streaming
https://spark.apache.org/streaming/[Spark Streaming] is a micro batching stream
processing framework built on top of Spark. HBase and Spark Streaming make great
companions in that HBase can help serve the following benefits alongside Spark
Streaming.
* A place to grab reference data or profile data on the fly
* A place to store counts or aggregates in a way that supports Spark Streaming's
promise of _only once processing_.
The link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
with Spark Streaming is similar to its normal Spark integration points, in that the following
commands are possible straight off a Spark Streaming DStream.
include::spark.adoc[tags=spark_base_functions]
.`bulkPut` Example with DStreams
====
Below is an example of bulkPut with DStreams. It is very close in feel to the RDD
bulk put.
[source, scala]
----
val sc = new SparkContext("local", "test")
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val ssc = new StreamingContext(sc, Milliseconds(200))
val rdd1 = ...
val rdd2 = ...
val queue = mutable.Queue[RDD[(Array[Byte], Array[(Array[Byte],
Array[Byte], Array[Byte])])]]()
queue += rdd1
queue += rdd2
val dStream = ssc.queueStream(queue)
dStream.hbaseBulkPut(
hbaseContext,
TableName.valueOf(tableName),
(putRecord) => {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
put
})
----
There are three inputs to the `hbaseBulkPut` function.
The hbaseContext that carries the configuration broadcast information link
to the HBase Connections in the executor, the table name of the table we are
putting data into, and a function that will convert a record in the DStream
into an HBase Put object.
====
== Bulk Load
There are two options for bulk loading data into HBase with Spark. There is the
basic bulk load functionality that will work for cases where your rows have
millions of columns and cases where your columns are not consolidated and
partitioned before the map side of the Spark bulk load process.
There is also a thin record bulk load option with Spark. This second option is
designed for tables that have less then 10k columns per row. The advantage
of this second option is higher throughput and less over-all load on the Spark
shuffle operation.
Both implementations work more or less like the MapReduce bulk load process in
that a partitioner partitions the rowkeys based on region splits and
the row keys are sent to the reducers in order, so that HFiles can be written
out directly from the reduce phase.
In Spark terms, the bulk load will be implemented around a Spark
`repartitionAndSortWithinPartitions` followed by a Spark `foreachPartition`.
First lets look at an example of using the basic bulk load functionality
.Bulk Loading Example
====
The following example shows bulk loading with Spark.
[source, scala]
----
val sc = new SparkContext("local", "test")
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val stagingFolder = ...
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), ...
rdd.hbaseBulkLoad(TableName.valueOf(tableName),
t => {
val rowKey = t._1
val family:Array[Byte] = t._2(0)._1
val qualifier = t._2(0)._2
val value = t._2(0)._3
val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
Seq((keyFamilyQualifier, value)).iterator
},
stagingFolder.getPath)
val load = new LoadIncrementalHFiles(config)
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
----
====
The `hbaseBulkLoad` function takes three required parameters:
. The table name of the table we intend to bulk load too
. A function that will convert a record in the RDD to a tuple key value par. With
the tuple key being a KeyFamilyQualifer object and the value being the cell value.
The KeyFamilyQualifer object will hold the RowKey, Column Family, and Column Qualifier.
The shuffle will partition on the RowKey but will sort by all three values.
. The temporary path for the HFile to be written out too
Following the Spark bulk load command, use the HBase's LoadIncrementalHFiles object
to load the newly created HFiles into HBase.
.Additional Parameters for Bulk Loading with Spark
You can set the following attributes with additional parameter options on hbaseBulkLoad.
* Max file size of the HFiles
* A flag to exclude HFiles from compactions
* Column Family settings for compression, bloomType, blockSize, and dataBlockEncoding
.Using Additional Parameters
====
[source, scala]
----
val sc = new SparkContext("local", "test")
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val stagingFolder = ...
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), ...
val familyHBaseWriterOptions = new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions]
val f1Options = new FamilyHFileWriteOptions("GZ", "ROW", 128, "PREFIX")
familyHBaseWriterOptions.put(Bytes.toBytes("columnFamily1"), f1Options)
rdd.hbaseBulkLoad(TableName.valueOf(tableName),
t => {
val rowKey = t._1
val family:Array[Byte] = t._2(0)._1
val qualifier = t._2(0)._2
val value = t._2(0)._3
val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
Seq((keyFamilyQualifier, value)).iterator
},
stagingFolder.getPath,
familyHBaseWriterOptions,
compactionExclude = false,
HConstants.DEFAULT_MAX_FILE_SIZE)
val load = new LoadIncrementalHFiles(config)
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
----
====
Now lets look at how you would call the thin record bulk load implementation
.Using thin record bulk load
====
[source, scala]
----
val sc = new SparkContext("local", "test")
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val stagingFolder = ...
val rdd = sc.parallelize(Array(
("1",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
("3",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), ...
rdd.hbaseBulkLoadThinRows(hbaseContext,
TableName.valueOf(tableName),
t => {
val rowKey = t._1
val familyQualifiersValues = new FamiliesQualifiersValues
t._2.foreach(f => {
val family:Array[Byte] = f._1
val qualifier = f._2
val value:Array[Byte] = f._3
familyQualifiersValues +=(family, qualifier, value)
})
(new ByteArrayWrapper(Bytes.toBytes(rowKey)), familyQualifiersValues)
},
stagingFolder.getPath,
new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions],
compactionExclude = false,
20)
val load = new LoadIncrementalHFiles(config)
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
----
====
Note that the big difference in using bulk load for thin rows is the function
returns a tuple with the first value being the row key and the second value
being an object of FamiliesQualifiersValues, which will contain all the
values for this row for all column families.
== SparkSQL/DataFrames
The link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
leverages
link:https://databricks.com/blog/2015/01/09/spark-sql-data-sources-api-unified-data-access-for-the-spark-platform.html[DataSource API]
(link:https://issues.apache.org/jira/browse/SPARK-3247[SPARK-3247])
introduced in Spark-1.2.0, which bridges the gap between simple HBase KV store and complex
relational SQL queries and enables users to perform complex data analytical work
on top of HBase using Spark. HBase Dataframe is a standard Spark Dataframe, and is able to
interact with any other data sources such as Hive, Orc, Parquet, JSON, etc.
The link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
applies critical techniques such as partition pruning, column pruning,
predicate pushdown and data locality.
To use the
link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
connector, users need to define the Catalog for the schema mapping
between HBase and Spark tables, prepare the data and populate the HBase table,
then load the HBase DataFrame. After that, users can do integrated query and access records
in HBase tables with SQL query. The following illustrates the basic procedure.
=== Define catalog
[source, scala]
----
def catalog = s"""{
       |"table":{"namespace":"default", "name":"table1"},
       |"rowkey":"key",
       |"columns":{
         |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
         |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
         |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
         |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
         |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
         |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
         |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
         |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
         |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
       |}
     |}""".stripMargin
----
Catalog defines a mapping between HBase and Spark tables. There are two critical parts of this catalog.
One is the rowkey definition and the other is the mapping between table column in Spark and
the column family and column qualifier in HBase. The above defines a schema for a HBase table
with name as table1, row key as key and a number of columns (col1 `-` col8). Note that the rowkey
also has to be defined in details as a column (col0), which has a specific cf (rowkey).
=== Save the DataFrame
[source, scala]
----
case class HBaseRecord(
col0: String,
col1: Boolean,
col2: Double,
col3: Float,
col4: Int,       
col5: Long,
col6: Short,
col7: String,
col8: Byte)
object HBaseRecord
{                                                                                                             
def apply(i: Int, t: String): HBaseRecord = {
val s = s"""row${"%03d".format(i)}"""       
HBaseRecord(s,
i % 2 == 0,
i.toDouble,
i.toFloat,  
i,
i.toLong,
i.toShort,  
s"String$i: $t",      
i.toByte)
}
}
val data = (0 to 255).map { i =>  HBaseRecord(i, "extra")}
sc.parallelize(data).toDF.write.options(
 Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
 .format("org.apache.hadoop.hbase.spark ")
 .save()
----
`data` prepared by the user is a local Scala collection which has 256 HBaseRecord objects.
`sc.parallelize(data)` function distributes `data` to form an RDD. `toDF` returns a DataFrame.
`write` function returns a DataFrameWriter used to write the DataFrame to external storage
systems (e.g. HBase here). Given a DataFrame with specified schema `catalog`, `save` function
will create an HBase table with 5 regions and save the DataFrame inside.
=== Load the DataFrame
[source, scala]
----
def withCatalog(cat: String): DataFrame = {
sqlContext
.read
.options(Map(HBaseTableCatalog.tableCatalog->cat))
.format("org.apache.hadoop.hbase.spark")
.load()
}
val df = withCatalog(catalog)
----
In withCatalog function, sqlContext is a variable of SQLContext, which is the entry point
for working with structured data (rows and columns) in Spark.
`read` returns a DataFrameReader that can be used to read data in as a DataFrame.
`option` function adds input options for the underlying data source to the DataFrameReader,
and `format` function specifies the input data source format for the DataFrameReader.
The `load()` function loads input in as a DataFrame. The date frame `df` returned
by `withCatalog` function could be used to access HBase table, such as 4.4 and 4.5.
=== Language Integrated Query
[source, scala]
----
val s = df.filter(($"col0" <= "row050" && $"col0" > "row040") ||
$"col0" === "row005" ||
$"col0" <= "row005")
.select("col0", "col1", "col4")
s.show
----
DataFrame can do various operations, such as join, sort, select, filter, orderBy and so on.
`df.filter` above filters rows using the given SQL expression. `select` selects a set of columns:
`col0`, `col1` and `col4`.
=== SQL Query
[source, scala]
----
df.registerTempTable("table1")
sqlContext.sql("select count(col1) from table1").show
----
`registerTempTable` registers `df` DataFrame as a temporary table using the table name `table1`.
The lifetime of this temporary table is tied to the SQLContext that was used to create `df`.
`sqlContext.sql` function allows the user to execute SQL queries.
=== Others
.Query with different timestamps
====
In HBaseSparkConf, four parameters related to timestamp can be set. They are TIMESTAMP,
MIN_TIMESTAMP, MAX_TIMESTAMP and MAX_VERSIONS respectively. Users can query records with
different timestamps or time ranges with MIN_TIMESTAMP and MAX_TIMESTAMP. In the meantime,
use concrete value instead of tsSpecified and oldMs in the examples below.
The example below shows how to load df DataFrame with different timestamps.
tsSpecified is specified by the user.
HBaseTableCatalog defines the HBase and Relation relation schema.
writeCatalog defines catalog for the schema mapping.
[source, scala]
----
val df = sqlContext.read
.options(Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseSparkConf.TIMESTAMP -> tsSpecified.toString))
.format("org.apache.hadoop.hbase.spark")
.load()
----
The example below shows how to load df DataFrame with different time ranges.
oldMs is specified by the user.
[source, scala]
----
val df = sqlContext.read
.options(Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseSparkConf.MIN_TIMESTAMP -> "0",
HBaseSparkConf.MAX_TIMESTAMP -> oldMs.toString))
.format("org.apache.hadoop.hbase.spark")
.load()
----
After loading df DataFrame, users can query data.
[source, scala]
----
df.registerTempTable("table")
sqlContext.sql("select count(col1) from table").show
----
====
.Native Avro support
====
The link:https://github.com/apache/hbase-connectors/tree/master/spark[hbase-spark integration]
connector supports different data formats like Avro, JSON, etc. The use case below
shows how spark supports Avro. Users can persist the Avro record into HBase directly. Internally,
the Avro schema is converted to a native Spark Catalyst data type automatically.
Note that both key-value parts in an HBase table can be defined in Avro format.
1) Define catalog for the schema mapping:
[source, scala]
----
def catalog = s"""{
|"table":{"namespace":"default", "name":"Avrotable"},
|"rowkey":"key",
|"columns":{
|"col0":{"cf":"rowkey", "col":"key", "type":"string"},
|"col1":{"cf":"cf1", "col":"col1", "type":"binary"}
|}
|}""".stripMargin
----
`catalog` is a schema for a HBase table named `Avrotable`. row key as key and
one column col1. The rowkey also has to be defined in details as a column (col0),
which has a specific cf (rowkey).
2) Prepare the Data:
[source, scala]
----
object AvroHBaseRecord {
val schemaString =
s"""{"namespace": "example.avro",
| "type": "record", "name": "User",
| "fields": [
| {"name": "name", "type": "string"},
| {"name": "favorite_number", "type": ["int", "null"]},
| {"name": "favorite_color", "type": ["string", "null"]},
| {"name": "favorite_array", "type": {"type": "array", "items": "string"}},
| {"name": "favorite_map", "type": {"type": "map", "values": "int"}}
| ] }""".stripMargin
val avroSchema: Schema = {
val p = new Schema.Parser
p.parse(schemaString)
}
def apply(i: Int): AvroHBaseRecord = {
val user = new GenericData.Record(avroSchema);
user.put("name", s"name${"%03d".format(i)}")
user.put("favorite_number", i)
user.put("favorite_color", s"color${"%03d".format(i)}")
val favoriteArray = new GenericData.Array[String](2, avroSchema.getField("favorite_array").schema())
favoriteArray.add(s"number${i}")
favoriteArray.add(s"number${i+1}")
user.put("favorite_array", favoriteArray)
import collection.JavaConverters._
val favoriteMap = Map[String, Int](("key1" -> i), ("key2" -> (i+1))).asJava
user.put("favorite_map", favoriteMap)
val avroByte = AvroSedes.serialize(user, avroSchema)
AvroHBaseRecord(s"name${"%03d".format(i)}", avroByte)
}
}
val data = (0 to 255).map { i =>
AvroHBaseRecord(i)
}
----
`schemaString` is defined first, then it is parsed to get `avroSchema`. `avroSchema` is used to
generate `AvroHBaseRecord`. `data` prepared by users is a local Scala collection
which has 256 `AvroHBaseRecord` objects.
3) Save DataFrame:
[source, scala]
----
sc.parallelize(data).toDF.write.options(
Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
.format("org.apache.spark.sql.execution.datasources.hbase")
.save()
----
Given a data frame with specified schema `catalog`, above will create an HBase table with 5
regions and save the data frame inside.
4) Load the DataFrame
[source, scala]
----
def avroCatalog = s"""{
|"table":{"namespace":"default", "name":"avrotable"},
|"rowkey":"key",
|"columns":{
|"col0":{"cf":"rowkey", "col":"key", "type":"string"},
|"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
|}
|}""".stripMargin
def withCatalog(cat: String): DataFrame = {
sqlContext
.read
.options(Map("avroSchema" -> AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog -> avroCatalog))
.format("org.apache.spark.sql.execution.datasources.hbase")
.load()
}
val df = withCatalog(catalog)
----
In `withCatalog` function, `read` returns a DataFrameReader that can be used to read data in as a DataFrame.
The `option` function adds input options for the underlying data source to the DataFrameReader.
There are two options: one is to set `avroSchema` as `AvroHBaseRecord.schemaString`, and one is to
set `HBaseTableCatalog.tableCatalog` as `avroCatalog`. The `load()` function loads input in as a DataFrame.
The date frame `df` returned by `withCatalog` function could be used to access the HBase table.
5) SQL Query
[source, scala]
----
df.registerTempTable("avrotable")
val c = sqlContext.sql("select count(1) from avrotable").
----
After loading df DataFrame, users can query data. registerTempTable registers df DataFrame
as a temporary table using the table name avrotable. `sqlContext.sql` function allows the
user to execute SQL queries.
====

View File

@ -724,6 +724,17 @@ Insure the JCE jars are on the classpath on both server and client systems.
You may also need to download the link:http://www.oracle.com/technetwork/java/javase/downloads/jce-6-download-429243.html[unlimited strength JCE policy files].
Uncompress and extract the downloaded file, and install the policy jars into _<java-home>/lib/security_.
[[trouble.client.masterregistry]]
=== Trouble shooting master registry issues
* For connectivity issues, usually an exception like "MasterRegistryFetchException: Exception making rpc to masters..." is logged in the client logs. The logging includes the
list of master end points that were attempted by the client. The bottom part of the stack trace should include the underlying reason. If you suspect connectivity
issues (ConnectionRefused?), make sure the master end points are accessible from client.
* If there is a suspicion of higher load on the masters due to hedging of RPCs, it can be controlled by either reducing the hedging fan out (via _hbase.rpc.hedged.fanout_) or
by restricting the set of masters that clients can access for the master registry purposes (via _hbase.masters_).
Refer to <<client.masterregistry>> and <<client_dependencies>> for more details.
[[trouble.mapreduce]]
== MapReduce
@ -870,9 +881,9 @@ Snapshots::
When you create a snapshot, HBase retains everything it needs to recreate the table's
state at that time of the snapshot. This includes deleted cells or expired versions.
For this reason, your snapshot usage pattern should be well-planned, and you should
prune snapshots that you no longer need. Snapshots are stored in `/hbase/.snapshots`,
prune snapshots that you no longer need. Snapshots are stored in `/hbase/.hbase-snapshot`,
and archives needed to restore snapshots are stored in
`/hbase/.archive/<_tablename_>/<_region_>/<_column_family_>/`.
`/hbase/archive/<_tablename_>/<_region_>/<_column_family_>/`.
*Do not* manage snapshots or archives manually via HDFS. HBase provides APIs and
HBase Shell commands for managing them. For more information, see <<ops.snapshots>>.
@ -1290,7 +1301,7 @@ If you have a DNS server, you can set `hbase.zookeeper.dns.interface` and `hbase
ZooKeeper is the cluster's "canary in the mineshaft". It'll be the first to notice issues if any so making sure its happy is the short-cut to a humming cluster.
See the link:https://wiki.apache.org/hadoop/ZooKeeper/Troubleshooting[ZooKeeper Operating Environment Troubleshooting] page.
See the link:https://cwiki.apache.org/confluence/display/HADOOP2/ZooKeeper+Troubleshooting[ZooKeeper Operating Environment Troubleshooting] page.
It has suggestions and tools for checking disk and networking performance; i.e.
the operating environment your ZooKeeper and HBase are running in.

View File

@ -162,7 +162,7 @@ HBase Private API::
[[hbase.binary.compatibility]]
.Binary Compatibility
When we say two HBase versions are compatible, we mean that the versions are wire and binary compatible. Compatible HBase versions means that clients can talk to compatible but differently versioned servers. It means too that you can just swap out the jars of one version and replace them with the jars of another, compatible version and all will just work. Unless otherwise specified, HBase point versions are (mostly) binary compatible. You can safely do rolling upgrades between binary compatible versions; i.e. across maintenance releases: e.g. from 1.2.4 to 1.2.6. See link:[Does compatibility between versions also mean binary compatibility?] discussion on the HBase dev mailing list.
When we say two HBase versions are compatible, we mean that the versions are wire and binary compatible. Compatible HBase versions means that clients can talk to compatible but differently versioned servers. It means too that you can just swap out the jars of one version and replace them with the jars of another, compatible version and all will just work. Unless otherwise specified, HBase point versions are (mostly) binary compatible. You can safely do rolling upgrades between binary compatible versions; i.e. across maintenance releases: e.g. from 1.4.4 to 1.4.6. See link:[Does compatibility between versions also mean binary compatibility?] discussion on the HBase dev mailing list.
[[hbase.rolling.upgrade]]
=== Rolling Upgrades
@ -180,7 +180,7 @@ The rolling-restart script will first gracefully stop and restart the master, an
[[hbase.rolling.restart]]
.Rolling Upgrade Between Versions that are Binary/Wire Compatible
Unless otherwise specified, HBase minor versions are binary compatible. You can do a <<hbase.rolling.upgrade>> between HBase point versions. For example, you can go to 1.2.4 from 1.2.6 by doing a rolling upgrade across the cluster replacing the 1.2.4 binary with a 1.2.6 binary.
Unless otherwise specified, HBase minor versions are binary compatible. You can do a <<hbase.rolling.upgrade>> between HBase point versions. For example, you can go to 1.4.4 from 1.4.6 by doing a rolling upgrade across the cluster replacing the 1.4.4 binary with a 1.4.6 binary.
In the minor version-particular sections below, we call out where the versions are wire/protocol compatible and in this case, it is also possible to do a <<hbase.rolling.upgrade>>.
@ -315,6 +315,50 @@ Quitting...
== Upgrade Paths
[[upgrade2.3]]
=== Upgrade from 2.0.x-2.2.x to 2.3+
There is no special consideration upgrading to hbase-2.3.x from earlier versions. From 2.2.x, it should be
rolling upgradeable. From 2.1.x or 2.0.x, you will need to clear the <<upgrade2.2>> hurdle first.
[[upgrade2.3_zookeeper]]
==== Upgraded ZooKeeper Dependency Version
Our dependency on Apache ZooKeeper has been upgraded to 3.5.7
(https://issues.apache.org/jira/browse/HBASE-24132[HBASE-24132]), as 3.4.x is EOL. The newer 3.5.x
client is compatible with the older 3.4.x server. However, if you're using HBase in stand-alone
mode and perform an in-place upgrade, there are some upgrade steps
https://cwiki.apache.org/confluence/display/ZOOKEEPER/Upgrade+FAQ[documented by the ZooKeeper community].
This doesn't impact a production deployment, but would impact a developer's local environment.
[[upgrade2.3_in-master-procedure-store-region]]
==== New In-Master Procedure Store
Of note, HBase 2.3.0 changes the in-Master Procedure Store implementation. It was a dedicated custom store
(see <<master.wal>>) to instead use a standard HBase Region (https://issues.apache.org/jira/browse/HBASE-23326[HBASE-23326]).
The migration from the old to new format is automatic run by the new 2.3.0 Master on startup. The old _MasterProcWALs_
dir which hosted the old custom implementation files in _${hbase.rootdir}_ is deleted on successful
migration. A new _MasterProc_ sub-directory replaces it to host the Store files and WALs for the new
Procedure Store in-Master Region. The in-Master Region is unusual in that it writes to an
alternate location at _${hbase.rootdir}/MasterProc_ rather than under _${hbase.rootdir}/data_ in the
filesystem and the special Procedure Store in-Master Region is hidden from all clients other than the active
Master itself. Otherwise, it is like any other with the Master process running flushes and compactions,
archiving WALs when over-flushed, and so on. Its files are readable by standard Region and Store file
tooling for triage and analysis as long as they are pointed to the appropriate location in the filesystem.
[[upgrade2.2]]
=== Upgrade from 2.0 or 2.1 to 2.2+
HBase 2.2+ uses a new Procedure form assiging/unassigning/moving Regions. It does not process HBase 2.1 and 2.0's Unassign/Assign Procedure types. Upgrade requires that we first drain the Master Procedure Store of old style Procedures before starting the new 2.2 Master. So you need to make sure that before you kill the old version (2.0 or 2.1) Master, there is no region in transition. And once the new version (2.2+) Master is up, you can rolling upgrade RegionServers one by one.
And there is a more safer way if you are running 2.1.1+ or 2.0.3+ cluster. It need four steps to upgrade Master.
. Shutdown both active and standby Masters (Your cluster will continue to server reads and writes without interruption).
. Set the property hbase.procedure.upgrade-to-2-2 to true in hbase-site.xml for the Master, and start only one Master, still using the 2.1.1+ (or 2.0.3+) version.
. Wait until the Master quits. Confirm that there is a 'READY TO ROLLING UPGRADE' message in the Master log as the cause of the shutdown. The Procedure Store is now empty.
. Start new Masters with the new 2.2+ version.
Then you can rolling upgrade RegionServers one by one. See link:https://issues.apache.org/jira/browse/HBASE-21075[HBASE-21075] for more details.
[[upgrade2.0]]
=== Upgrading from 1.x to 2.x
@ -332,7 +376,11 @@ As noted in the section <<basic.prerequisites>>, HBase 2.0+ requires a minimum o
.HBCK must match HBase server version
You *must not* use an HBase 1.x version of HBCK against an HBase 2.0+ cluster. HBCK is strongly tied to the HBase server version. Using the HBCK tool from an earlier release against an HBase 2.0+ cluster will destructively alter said cluster in unrecoverable ways.
As of HBase 2.0, HBCK is a read-only tool that can report the status of some non-public system internals. You should not rely on the format nor content of these internals to remain consistent across HBase releases.
As of HBase 2.0, HBCK (A.K.A _HBCK1_ or _hbck1_) is a read-only tool that can report the status of some non-public system internals but will often misread state because it does not understand the workings of hbase2.
To read about HBCK's replacement, see <<HBCK2>> in <<ops_mgt>>.
IMPORTANT: Related, before you upgrade, ensure that _hbck1_ reports no `INCONSISTENCIES`. Fixing hbase1-type inconsistencies post-upgrade is an involved process.
////
Link to a ref guide section on HBCK in 2.0 that explains use and calls out the inability of clients and server sides to detect version of each other.
@ -614,6 +662,18 @@ Performance is also an area that is now under active review so look forward to
improvement in coming releases (See
link:https://issues.apache.org/jira/browse/HBASE-20188[HBASE-20188 TESTING Performance]).
[[upgrade2.0.it.kerberos]]
.Integration Tests and Kerberos
Integration Tests (`IntegrationTests*`) used to rely on the Kerberos credential cache
for authentication against secured clusters. This used to lead to tests failing due
to authentication failures when the tickets in the credential cache expired.
As of hbase-2.0.0 (and hbase-1.3.0+), the integration test clients will make use
of the configuration properties `hbase.client.keytab.file` and
`hbase.client.kerberos.principal`. They are required. The clients will perform a
login from the configured keytab file and automatically refresh the credentials
in the background for the process lifetime (See
link:https://issues.apache.org/jira/browse/HBASE-16231[HBASE-16231]).
[[upgrade2.0.compaction.throughput.limit]]
.Default Compaction Throughput
HBase 2.x comes with default limits to the speed at which compactions can execute. This
@ -706,6 +766,7 @@ rolling upgrade of a 1.4 cluster.
.Pre-Requirements
* Upgrade to the latest 1.4.x release. Pre 1.4 releases may also work but are not tested, so please upgrade to 1.4.3+ before upgrading to 2.x, unless you are an expert and familiar with the region assignment and crash processing. See the section <<upgrade1.4>> on how to upgrade to 1.4.x.
* Make sure that the zk-less assignment is enabled, i.e, set `hbase.assignment.usezk` to `false`. This is the most important thing. It allows the 1.x master to assign/unassign regions to/from 2.x region servers. See the release note section of link:https://issues.apache.org/jira/browse/HBASE-11059[HBASE-11059] on how to migrate from zk based assignment to zk less assignment.
* Before you upgrade, ensure that _hbck1_ reports no `INCONSISTENCIES`. Fixing hbase1-type inconsistencies post-upgrade is an involved process.
* We have tested rolling upgrading from 1.4.3 to 2.1.0, but it should also work if you want to upgrade to 2.0.x.
.Instructions
@ -726,6 +787,7 @@ NOTE: If you have success running this prescription, please notify the dev list
To upgrade an existing HBase 1.x cluster, you should:
* Ensure that _hbck1_ reports no `INCONSISTENCIES`. Fixing hbase1-type inconsistencies post-upgrade is an involved process. Fix all _hbck1_ complaints before proceeding.
* Clean shutdown of existing 1.x cluster
* Update coprocessors
* Upgrade Master roles first
@ -764,6 +826,11 @@ Notes:
Doing a raw scan will now return results that have expired according to TTL settings.
[[upgrade1.3]]
=== Upgrading from pre-1.3 to 1.3+
If running Integration Tests under Kerberos, see <<upgrade2.0.it.kerberos>>.
[[upgrade1.0]]
=== Upgrading to 1.x

View File

@ -137,7 +137,7 @@ Just make sure to set `HBASE_MANAGES_ZK` to `false` if you want it to stay
For more information about running a distinct ZooKeeper cluster, see the ZooKeeper link:https://zookeeper.apache.org/doc/current/zookeeperStarted.html[Getting
Started Guide].
Additionally, see the link:https://wiki.apache.org/hadoop/ZooKeeper/FAQ#A7[ZooKeeper Wiki] or the link:https://zookeeper.apache.org/doc/r3.3.3/zookeeperAdmin.html#sc_zkMulitServerSetup[ZooKeeper
Additionally, see the link:https://cwiki.apache.org/confluence/display/HADOOP2/ZooKeeper+FAQ#ZooKeeperFAQ-7[ZooKeeper Wiki] or the link:https://zookeeper.apache.org/doc/r3.4.10/zookeeperAdmin.html#sc_zkMulitServerSetup[ZooKeeper
documentation] for more information on ZooKeeper sizing.
[[zk.sasl.auth]]

View File

@ -38,6 +38,7 @@
:experimental:
:source-language: java
:leveloffset: 0
:stem:
// Logo for HTML -- doesn't render in PDF
ifdef::backend-html5[]
@ -64,11 +65,13 @@ include::_chapters/mapreduce.adoc[]
include::_chapters/security.adoc[]
include::_chapters/architecture.adoc[]
include::_chapters/hbase_mob.adoc[]
include::_chapters/snapshot_scanner.adoc[]
include::_chapters/inmemory_compaction.adoc[]
include::_chapters/offheap_read_write.adoc[]
include::_chapters/hbase_apis.adoc[]
include::_chapters/external_apis.adoc[]
include::_chapters/thrift_filter_language.adoc[]
include::_chapters/spark.adoc[]
include::_chapters/cp.adoc[]
include::_chapters/performance.adoc[]
include::_chapters/profiler.adoc[]
@ -82,12 +85,12 @@ include::_chapters/pv2.adoc[]
include::_chapters/amv2.adoc[]
include::_chapters/zookeeper.adoc[]
include::_chapters/community.adoc[]
include::_chapters/hbtop.adoc[]
= Appendix
include::_chapters/appendix_contributing_to_documentation.adoc[]
include::_chapters/faq.adoc[]
include::_chapters/hbck_in_depth.adoc[]
include::_chapters/appendix_acl_matrix.adoc[]
include::_chapters/compression.adoc[]
include::_chapters/sql.adoc[]

View File

@ -82,7 +82,7 @@ NOTE:This is not true _across rows_ for multirow batch mutations.
A scan is *not* a consistent view of a table. Scans do *not* exhibit _snapshot isolation_.
Rather, scans have the following properties:
. Any row returned by the scan will be a consistent view (i.e. that version of the complete row existed at some point in time)footnoteref[consistency,A consistent view is not guaranteed intra-row scanning -- i.e. fetching a portion of a row in one RPC then going back to fetch another portion of the row in a subsequent RPC. Intra-row scanning happens when you set a limit on how many values to return per Scan#next (See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setBatch(int)"[Scan#setBatch(int)]).]
. Any row returned by the scan will be a consistent view (i.e. that version of the complete row existed at some point in time)footnoteref[consistency,A consistent view is not guaranteed intra-row scanning -- i.e. fetching a portion of a row in one RPC then going back to fetch another portion of the row in a subsequent RPC. Intra-row scanning happens when you set a limit on how many values to return per Scan#next (See link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setBatch(int)"[Scan#setBatch(int)]).]
. A scan will always reflect a view of the data _at least as new as_ the beginning of the scan. This satisfies the visibility guarantees enumerated below.
.. For example, if client A writes data X and then communicates via a side channel to client B, any scans started by client B will contain data at least as new as X.
.. A scan _must_ reflect all mutations committed prior to the construction of the scanner, and _may_ reflect some mutations committed subsequent to the construction of the scanner.
@ -115,4 +115,4 @@ All of the above guarantees must be possible within Apache HBase. For users who
== More Information
For more information, see the link:book.html#client[client architecture] and link:book.html#datamodel[data model] sections in the Apache HBase Reference Guide.
For more information, see the link:book.html#client[client architecture] and link:book.html#datamodel[data model] sections in the Apache HBase Reference Guide.

View File

@ -20,3 +20,4 @@ under the License.
= Bulk Loads in Apache HBase (TM)
This page has been retired. The contents have been moved to the link:book.html#arch.bulk.load[Bulk Loading] section in the Reference Guide.

View File

@ -29,11 +29,11 @@ encryption software, to see if this is permitted. See the
link:http://www.wassenaar.org/[Wassenaar Arrangement] for more
information.
The U.S. Government Department of Commerce, Bureau of Industry and Security
(BIS), has classified this software as Export Commodity Control Number (ECCN)
5D002.C.1, which includes information security software using or performing
The U.S. Government Department of Commerce, Bureau of Industry and Security
(BIS), has classified this software as Export Commodity Control Number (ECCN)
5D002.C.1, which includes information security software using or performing
cryptographic functions with asymmetric algorithms. The form and manner of this
Apache Software Foundation distribution makes it eligible for export under the
Apache Software Foundation distribution makes it eligible for export under the
License Exception ENC Technology Software Unrestricted (TSU) exception (see the
BIS Export Administration Regulations, Section 740.13) for both object code and
source code.

View File

@ -20,7 +20,7 @@ under the License.
= Apache HBase&#153; Home
.Welcome to Apache HBase(TM)
link:http://www.apache.org/[Apache HBase(TM)] is the link:http://hadoop.apache.org[Hadoop] database, a distributed, scalable, big data store.
link:https://www.apache.org/[Apache HBase(TM)] is the link:https://hadoop.apache.org[Hadoop] database, a distributed, scalable, big data store.
.When Would I Use Apache HBase?
Use Apache HBase when you need random, realtime read/write access to your Big Data. +

View File

@ -20,13 +20,13 @@ under the License.
= Apache HBase (TM) Metrics
== Introduction
Apache HBase (TM) emits Hadoop link:http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html[metrics].
Apache HBase (TM) emits Hadoop link:https://hadoop.apache.org/core/docs/stable/api/org/apache/hadoop/metrics/package-summary.html[metrics].
== Setup
First read up on Hadoop link:http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html[metrics].
First read up on Hadoop link:https://hadoop.apache.org/core/docs/stable/api/org/apache/hadoop/metrics/package-summary.html[metrics].
If you are using ganglia, the link:http://wiki.apache.org/hadoop/GangliaMetrics[GangliaMetrics] wiki page is useful read.
If you are using ganglia, the link:https://cwiki.apache.org/confluence/display/HADOOP2/GangliaMetrics[GangliaMetrics] wiki page is useful read.
To have HBase emit metrics, edit `$HBASE_HOME/conf/hadoop-metrics.properties` and enable metric 'contexts' per plugin. As of this writing, hadoop supports *file* and *ganglia* plugins. Yes, the hbase metrics files is named hadoop-metrics rather than _hbase-metrics_ because currently at least the hadoop metrics system has the properties filename hardcoded. Per metrics _context_, comment out the NullContext and enable one or more plugins instead.
@ -41,9 +41,9 @@ The _jvm_ context is useful for long-term stats on running hbase jvms -- memory
== Using with JMX
In addition to the standard output contexts supported by the Hadoop
metrics package, you can also export HBase metrics via Java Management
Extensions (JMX). This will allow viewing HBase stats in JConsole or
In addition to the standard output contexts supported by the Hadoop
metrics package, you can also export HBase metrics via Java Management
Extensions (JMX). This will allow viewing HBase stats in JConsole or
any other JMX client.
=== Enable HBase stats collection
@ -67,7 +67,7 @@ rpc.period=60
=== Setup JMX Remote Access
For remote access, you will need to configure JMX remote passwords and access profiles. Create the files:
`$HBASE_HOME/conf/jmxremote.passwd` (set permissions
`$HBASE_HOME/conf/jmxremote.passwd` (set permissions
to 600):: +
----
monitorRole monitorpass
@ -98,4 +98,5 @@ After restarting the processes you want to monitor, you should now be able to ru
== Understanding HBase Metrics
For more information on understanding HBase metrics, see the link:book.html#hbase_metrics[metrics section] in the Apache HBase Reference Guide.
For more information on understanding HBase metrics, see the link:book.html#hbase_metrics[metrics section] in the Apache HBase Reference Guide.

View File

@ -57,7 +57,7 @@ October 25th, 2012:: link:http://www.meetup.com/HBase-NYC/events/81728932/[Strat
September 11th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/80621872/[Contributor's Pow-Wow at HortonWorks HQ.]
August 8th, 2012:: link:http://www.apache.org/dyn/closer.cgi/hbase/[Apache HBase 0.94.1 is available for download]
August 8th, 2012:: link:https://www.apache.org/dyn/closer.lua/hbase/[Apache HBase 0.94.1 is available for download]
June 15th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/59829652/[Birds-of-a-feather] in San Jose, day after:: link:http://hadoopsummit.org[Hadoop Summit]
@ -69,9 +69,9 @@ March 27th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/56021562/[Me
January 19th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/46702842/[Meetup @ EBay]
January 23rd, 2012:: Apache HBase 0.92.0 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
January 23rd, 2012:: Apache HBase 0.92.0 released. link:https://www.apache.org/dyn/closer.lua/hbase/[Download it!]
December 23rd, 2011:: Apache HBase 0.90.5 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
December 23rd, 2011:: Apache HBase 0.90.5 released. link:https://www.apache.org/dyn/closer.lua/hbase/[Download it!]
November 29th, 2011:: link:http://www.meetup.com/hackathon/events/41025972/[Developer Pow-Wow in SF] at Salesforce HQ
@ -83,9 +83,9 @@ June 30th, 2011:: link:http://www.meetup.com/hbaseusergroup/events/20572251/[HBa
June 8th, 2011:: link:http://berlinbuzzwords.de/wiki/hbase-workshop-and-hackathon[HBase Hackathon] in Berlin to coincide with:: link:http://berlinbuzzwords.de/[Berlin Buzzwords]
May 19th, 2011: Apache HBase 0.90.3 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
May 19th, 2011: Apache HBase 0.90.3 released. link:https://www.apache.org/dyn/closer.lua/hbase/[Download it!]
April 12th, 2011: Apache HBase 0.90.2 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
April 12th, 2011: Apache HBase 0.90.2 released. link:https://www.apache.org/dyn/closer.lua/hbase/[Download it!]
March 21st, 2011:: link:http://www.meetup.com/hackathon/events/16770852/[HBase 0.92 Hackathon at StumbleUpon, SF]
February 22nd, 2011:: link:http://www.meetup.com/hbaseusergroup/events/16492913/[HUG12: February HBase User Group at StumbleUpon SF]
@ -97,7 +97,7 @@ October 12th, 2010:: HBase-related presentations by core contributors and users
October 11th, 2010:: link:http://www.meetup.com/hbaseusergroup/calendar/14606174/[HUG-NYC: HBase User Group NYC Edition] (Night before Hadoop World)
June 30th, 2010:: link:http://www.meetup.com/hbaseusergroup/calendar/13562846/[Apache HBase Contributor Workshop] (Day after Hadoop Summit)
May 10th, 2010:: Apache HBase graduates from Hadoop sub-project to Apache Top Level Project
May 10th, 2010:: Apache HBase graduates from Hadoop sub-project to Apache Top Level Project
April 19, 2010:: Signup for link:http://www.meetup.com/hbaseusergroup/calendar/12689490/[HBase User Group Meeting, HUG10] hosted by Trend Micro
@ -105,7 +105,7 @@ March 10th, 2010:: link:http://www.meetup.com/hbaseusergroup/calendar/12689351/[
January 27th, 2010:: Sign up for the link:http://www.meetup.com/hbaseusergroup/calendar/12241393/[HBase User Group Meeting, HUG8], at StumbleUpon in SF
September 8th, 2010:: Apache HBase 0.20.0 is faster, stronger, slimmer, and sweeter tasting than any previous Apache HBase release. Get it off the link:http://www.apache.org/dyn/closer.cgi/hbase/[Releases] page.
September 8th, 2010:: Apache HBase 0.20.0 is faster, stronger, slimmer, and sweeter tasting than any previous Apache HBase release. Get it off the link:https://www.apache.org/dyn/closer.lua/hbase/[Releases] page.
November 2-6th, 2009:: link:http://dev.us.apachecon.com/c/acus2009/[ApacheCon] in Oakland. The Apache Foundation will be celebrating its 10th anniversary in beautiful Oakland by the Bay. Lots of good talks and meetups including an HBase presentation by a couple of the lads.
@ -118,3 +118,4 @@ June, 2009:: HBase at HadoopSummit2009 and at NOSQL: See the link:https://hbase
March 3rd, 2009 :: HUG6 -- link:http://www.meetup.com/hbaseusergroup/calendar/9764004/[HBase User Group 6]
January 30th, 2009:: LA Hbackathon: link:http://www.meetup.com/hbasela/calendar/9450876/[HBase January Hackathon Los Angeles] at link:http://streamy.com[Streamy] in Manhattan Beach

View File

@ -20,3 +20,4 @@ under the License.
= Running Apache HBase (TM) in pseudo-distributed mode
This page has been retired. The contents have been moved to the link:book.html#distributed[Distributed Operation: Pseudo- and Fully-distributed modes] section in the Reference Guide.

View File

@ -24,3 +24,4 @@ HBase: The Definitive Guide:: link:http://shop.oreilly.com/product/0636920014348
HBase In Action:: link:http://www.manning.com/dimidukkhurana[HBase In Action] By Nick Dimiduk and Amandeep Khurana. Publisher: Manning, MEAP Began: January 2012, Softbound print: Fall 2012, Pages: 350.
HBase Administration Cookbook:: link:http://www.packtpub.com/hbase-administration-for-optimum-database-performance-cookbook/book[HBase Administration Cookbook] by Yifeng Jiang. Publisher: PACKT Publishing, Release: Expected August 2012, Pages: 335.

View File

@ -19,11 +19,11 @@ under the License.
= Apache HBase(TM) Sponsors
First off, thanks to link:http://www.apache.org/foundation/thanks.html[all who sponsor] our parent, the Apache Software Foundation.
First off, thanks to link:https://www.apache.org/foundation/thanks.html[all who sponsor] our parent, the Apache Software Foundation.
The below companies have been gracious enough to provide their commerical tool offerings free of charge to the Apache HBase(TM) project.
* The crew at link:http://www.ej-technologies.com/[ej-technologies] have been letting us use link:http://www.ej-technologies.com/products/jprofiler/overview.html[JProfiler] for years now.
* The crew at link:http://www.ej-technologies.com/[ej-technologies] have been letting us use link:http://www.ej-technologies.com/products/jprofiler/overview.html[JProfiler] for years now.
* The lads at link:http://headwaysoftware.com/[headway software] have given us a license for link:http://headwaysoftware.com/products/?code=Restructure101[Restructure101] so we can untangle our interdependency mess.
@ -32,4 +32,5 @@ The below companies have been gracious enough to provide their commerical tool o
* Thank you to Boris at link:http://www.vectorportal.com/[Vector Portal] for granting us a license on the image on which our logo is based.
== Sponsoring the Apache Software Foundation">
To contribute to the Apache Software Foundation, a good idea in our opinion, see the link:http://www.apache.org/foundation/sponsorship.html[ASF Sponsorship] page.
To contribute to the Apache Software Foundation, a good idea in our opinion, see the link:https://www.apache.org/foundation/sponsorship.html[ASF Sponsorship] page.