HBASE-12922 Post-asciidoc conversion fix-ups part 2 <Lars Francke>

This commit is contained in:
Misty Stanley-Jones 2015-01-30 08:57:45 +10:00
parent eaacc5a0ce
commit a7816d88bd
19 changed files with 2318 additions and 2436 deletions

File diff suppressed because it is too large Load Diff

View File

@ -30,14 +30,14 @@
[[casestudies.overview]]
== Overview
This chapter will describe a variety of performance and troubleshooting case studies that can provide a useful blueprint on diagnosing Apache HBase cluster issues.
This chapter will describe a variety of performance and troubleshooting case studies that can provide a useful blueprint on diagnosing Apache HBase cluster issues.
For more information on Performance and Troubleshooting, see <<performance,performance>> and <<trouble,trouble>>.
For more information on Performance and Troubleshooting, see <<performance>> and <<trouble>>.
[[casestudies.schema]]
== Schema Design
See the schema design case studies here: <<schema.casestudies,schema.casestudies>>
See the schema design case studies here: <<schema.casestudies>>
[[casestudies.perftroub]]
== Performance/Troubleshooting
@ -49,16 +49,18 @@ See the schema design case studies here: <<schema.casestudies,schema.casestudies
Following a scheduled reboot, one data node began exhibiting unusual behavior.
Routine MapReduce jobs run against HBase tables which regularly completed in five or six minutes began taking 30 or 40 minutes to finish.
These jobs were consistently found to be waiting on map and reduce tasks assigned to the troubled data node (e.g., the slow map tasks all had the same Input Split). The situation came to a head during a distributed copy, when the copy was severely prolonged by the lagging node.
These jobs were consistently found to be waiting on map and reduce tasks assigned to the troubled data node (e.g., the slow map tasks all had the same Input Split). The situation came to a head during a distributed copy, when the copy was severely prolonged by the lagging node.
==== Hardware
* .Datanodes:Two 12-core processors
.Datanodes:
* Two 12-core processors
* Six Enerprise SATA disks
* 24GB of RAM
* Two bonded gigabit NICs
* .Network:10 Gigabit top-of-rack switches
.Network:
* 10 Gigabit top-of-rack switches
* 20 Gigabit bonded interconnects between racks.
==== Hypotheses
@ -68,61 +70,61 @@ These jobs were consistently found to be waiting on map and reduce tasks assigne
We hypothesized that we were experiencing a familiar point of pain: a "hot spot" region in an HBase table, where uneven key-space distribution can funnel a huge number of requests to a single HBase region, bombarding the RegionServer process and cause slow response time.
Examination of the HBase Master status page showed that the number of HBase requests to the troubled node was almost zero.
Further, examination of the HBase logs showed that there were no region splits, compactions, or other region transitions in progress.
This effectively ruled out a "hot spot" as the root cause of the observed slowness.
This effectively ruled out a "hot spot" as the root cause of the observed slowness.
===== HBase Region With Non-Local Data
Our next hypothesis was that one of the MapReduce tasks was requesting data from HBase that was not local to the datanode, thus forcing HDFS to request data blocks from other servers over the network.
Examination of the datanode logs showed that there were very few blocks being requested over the network, indicating that the HBase region was correctly assigned, and that the majority of the necessary data was located on the node.
This ruled out the possibility of non-local data causing a slowdown.
Our next hypothesis was that one of the MapReduce tasks was requesting data from HBase that was not local to the DataNode, thus forcing HDFS to request data blocks from other servers over the network.
Examination of the DataNode logs showed that there were very few blocks being requested over the network, indicating that the HBase region was correctly assigned, and that the majority of the necessary data was located on the node.
This ruled out the possibility of non-local data causing a slowdown.
===== Excessive I/O Wait Due To Swapping Or An Over-Worked Or Failing Hard Disk
After concluding that the Hadoop and HBase were not likely to be the culprits, we moved on to troubleshooting the datanode's hardware.
After concluding that the Hadoop and HBase were not likely to be the culprits, we moved on to troubleshooting the DataNode's hardware.
Java, by design, will periodically scan its entire memory space to do garbage collection.
If system memory is heavily overcommitted, the Linux kernel may enter a vicious cycle, using up all of its resources swapping Java heap back and forth from disk to RAM as Java tries to run garbage collection.
Further, a failing hard disk will often retry reads and/or writes many times before giving up and returning an error.
This can manifest as high iowait, as running processes wait for reads and writes to complete.
Finally, a disk nearing the upper edge of its performance envelope will begin to cause iowait as it informs the kernel that it cannot accept any more data, and the kernel queues incoming data into the dirty write pool in memory.
However, using `vmstat(1)` and `free(1)`, we could see that no swap was being used, and the amount of disk IO was only a few kilobytes per second.
However, using `vmstat(1)` and `free(1)`, we could see that no swap was being used, and the amount of disk IO was only a few kilobytes per second.
===== Slowness Due To High Processor Usage
Next, we checked to see whether the system was performing slowly simply due to very high computational load. `top(1)` showed that the system load was higher than normal, but `vmstat(1)` and `mpstat(1)` showed that the amount of processor being used for actual computation was low.
Next, we checked to see whether the system was performing slowly simply due to very high computational load. `top(1)` showed that the system load was higher than normal, but `vmstat(1)` and `mpstat(1)` showed that the amount of processor being used for actual computation was low.
===== Network Saturation (The Winner)
Since neither the disks nor the processors were being utilized heavily, we moved on to the performance of the network interfaces.
The datanode had two gigabit ethernet adapters, bonded to form an active-standby interface. `ifconfig(8)` showed some unusual anomalies, namely interface errors, overruns, framing errors.
While not unheard of, these kinds of errors are exceedingly rare on modern hardware which is operating as it should:
The DataNode had two gigabit ethernet adapters, bonded to form an active-standby interface. `ifconfig(8)` showed some unusual anomalies, namely interface errors, overruns, framing errors.
While not unheard of, these kinds of errors are exceedingly rare on modern hardware which is operating as it should:
----
$ /sbin/ifconfig bond0
bond0 Link encap:Ethernet HWaddr 00:00:00:00:00:00
bond0 Link encap:Ethernet HWaddr 00:00:00:00:00:00
inet addr:10.x.x.x Bcast:10.x.x.255 Mask:255.255.255.0
UP BROADCAST RUNNING MASTER MULTICAST MTU:1500 Metric:1
RX packets:2990700159 errors:12 dropped:0 overruns:1 frame:6 <--- Look Here! Errors!
TX packets:3443518196 errors:0 dropped:0 overruns:0 carrier:0
collisions:0 txqueuelen:0
collisions:0 txqueuelen:0
RX bytes:2416328868676 (2.4 TB) TX bytes:3464991094001 (3.4 TB)
----
These errors immediately lead us to suspect that one or more of the ethernet interfaces might have negotiated the wrong line speed.
This was confirmed both by running an ICMP ping from an external host and observing round-trip-time in excess of 700ms, and by running `ethtool(8)` on the members of the bond interface and discovering that the active interface was operating at 100Mbs/, full duplex.
This was confirmed both by running an ICMP ping from an external host and observing round-trip-time in excess of 700ms, and by running `ethtool(8)` on the members of the bond interface and discovering that the active interface was operating at 100Mbs/, full duplex.
----
$ sudo ethtool eth0
Settings for eth0:
Supported ports: [ TP ]
Supported link modes: 10baseT/Half 10baseT/Full
100baseT/Half 100baseT/Full
1000baseT/Full
Supported link modes: 10baseT/Half 10baseT/Full
100baseT/Half 100baseT/Full
1000baseT/Full
Supports auto-negotiation: Yes
Advertised link modes: 10baseT/Half 10baseT/Full
100baseT/Half 100baseT/Full
1000baseT/Full
Advertised link modes: 10baseT/Half 10baseT/Full
100baseT/Half 100baseT/Full
1000baseT/Full
Advertised pause frame use: No
Advertised auto-negotiation: Yes
Link partner advertised link modes: Not reported
@ -141,28 +143,28 @@ Current message level: 0x00000003 (3)
Link detected: yes
----
In normal operation, the ICMP ping round trip time should be around 20ms, and the interface speed and duplex should read, "1000MB/s", and, "Full", respectively.
In normal operation, the ICMP ping round trip time should be around 20ms, and the interface speed and duplex should read, "1000MB/s", and, "Full", respectively.
==== Resolution
After determining that the active ethernet adapter was at the incorrect speed, we used the `ifenslave(8)` command to make the standby interface the active interface, which yielded an immediate improvement in MapReduce performance, and a 10 times improvement in network throughput:
After determining that the active ethernet adapter was at the incorrect speed, we used the `ifenslave(8)` command to make the standby interface the active interface, which yielded an immediate improvement in MapReduce performance, and a 10 times improvement in network throughput:
On the next trip to the datacenter, we determined that the line speed issue was ultimately caused by a bad network cable, which was replaced.
On the next trip to the datacenter, we determined that the line speed issue was ultimately caused by a bad network cable, which was replaced.
[[casestudies.perf.1]]
=== Case Study #2 (Performance Research 2012)
Investigation results of a self-described "we're not sure what's wrong, but it seems slow" problem. link:http://gbif.blogspot.com/2012/03/hbase-performance-evaluation-continued.html
Investigation results of a self-described "we're not sure what's wrong, but it seems slow" problem. http://gbif.blogspot.com/2012/03/hbase-performance-evaluation-continued.html
[[casestudies.perf.2]]
=== Case Study #3 (Performance Research 2010))
Investigation results of general cluster performance from 2010.
Although this research is on an older version of the codebase, this writeup is still very useful in terms of approach. link:http://hstack.org/hbase-performance-testing/
Although this research is on an older version of the codebase, this writeup is still very useful in terms of approach. http://hstack.org/hbase-performance-testing/
[[casestudies.max.transfer.threads]]
=== Case Study #4 (max.transfer.threads Config)
Case study of configuring `max.transfer.threads` (previously known as `xcievers`) and diagnosing errors from misconfigurations. link:http://www.larsgeorge.com/2012/03/hadoop-hbase-and-xceivers.html
Case study of configuring `max.transfer.threads` (previously known as `xcievers`) and diagnosing errors from misconfigurations. http://www.larsgeorge.com/2012/03/hadoop-hbase-and-xceivers.html
See also <<dfs.datanode.max.transfer.threads,dfs.datanode.max.transfer.threads>>.
See also <<dfs.datanode.max.transfer.threads>>.

View File

@ -27,8 +27,8 @@
:icons: font
:experimental:
This chapter expands upon the <<getting_started,getting started>> chapter to further explain configuration of Apache HBase.
Please read this chapter carefully, especially <<basic.prerequisites,basic.prerequisites>> to ensure that your HBase testing and deployment goes smoothly, and prevent data loss.
This chapter expands upon the <<getting_started>> chapter to further explain configuration of Apache HBase.
Please read this chapter carefully, especially the <<basic.prerequisites,Basic Prerequisites>> to ensure that your HBase testing and deployment goes smoothly, and prevent data loss.
== Configuration Files
Apache HBase uses the same configuration system as Apache Hadoop.
@ -41,8 +41,7 @@ _backup-masters_::
_hadoop-metrics2-hbase.properties_::
Used to connect HBase Hadoop's Metrics2 framework.
See the link:http://wiki.apache.org/hadoop/HADOOP-6728-MetricsV2[Hadoop Wiki
entry] for more information on Metrics2.
See the link:http://wiki.apache.org/hadoop/HADOOP-6728-MetricsV2[Hadoop Wiki entry] for more information on Metrics2.
Contains only commented-out examples by default.
_hbase-env.cmd_ and _hbase-env.sh_::
@ -51,7 +50,7 @@ _hbase-env.cmd_ and _hbase-env.sh_::
_hbase-policy.xml_::
The default policy configuration file used by RPC servers to make authorization decisions on client requests.
Only used if HBase security (<<security,security>>) is enabled.
Only used if HBase <<security,security>> is enabled.
_hbase-site.xml_::
The main HBase configuration file.
@ -71,17 +70,16 @@ _regionservers_::
[TIP]
====
When you edit XML, it is a good idea to use an XML-aware editor to be sure that your syntax is correct and your XML is well-formed.
You can also use the +xmllint+ utility to check that your XML is well-formed.
By default, +xmllint+ re-flows and prints the XML to standard output.
To check for well-formedness and only print output if errors exist, use the command +xmllint -noout
filename.xml+.
You can also use the `xmllint` utility to check that your XML is well-formed.
By default, `xmllint` re-flows and prints the XML to standard output.
To check for well-formedness and only print output if errors exist, use the command `xmllint -noout filename.xml`.
====
.Keep Configuration In Sync Across the Cluster
[WARNING]
====
When running in distributed mode, after you make an edit to an HBase configuration, make sure you copy the content of the _conf/_ directory to all nodes of the cluster.
HBase will not do this for you.
Use +rsync+, +scp+, or another secure mechanism for copying the configuration files to your nodes.
Use `rsync`, `scp`, or another secure mechanism for copying the configuration files to your nodes.
For most configuration, a restart is needed for servers to pick up changes An exception is dynamic configuration.
to be described later below.
====
@ -89,8 +87,9 @@ to be described later below.
[[basic.prerequisites]]
== Basic Prerequisites
This section lists required services and some required system configuration.
This section lists required services and some required system configuration.
[[java]]
.Java
[cols="1,1,1,4", options="header"]
|===
@ -107,9 +106,9 @@ This section lists required services and some required system configuration.
|0.98
|yes
|yes
|Running with JDK 8 works but is not well tested. Building with JDK 8 would require removal of the
deprecated `remove()` method of the `PoolMap` class and is under consideration. See
link:https://issues.apache.org/jira/browse/HBASE-7608[HBASE-7608] for more information about JDK 8
|Running with JDK 8 works but is not well tested. Building with JDK 8 would require removal of the
deprecated `remove()` method of the `PoolMap` class and is under consideration. See
link:https://issues.apache.org/jira/browse/HBASE-7608[HBASE-7608] for more information about JDK 8
support.
|0.96
@ -127,27 +126,27 @@ NOTE: In HBase 0.98.5 and newer, you must set `JAVA_HOME` on each node of your c
.Operating System Utilities
ssh::
HBase uses the Secure Shell (ssh) command and utilities extensively to communicate between cluster nodes. Each server in the cluster must be running +ssh+ so that the Hadoop and HBase daemons can be managed. You must be able to connect to all nodes via SSH, including the local node, from the Master as well as any backup Master, using a shared key rather than a password. You can see the basic methodology for such a set-up in Linux or Unix systems at <<passwordless.ssh.quickstart,passwordless.ssh.quickstart>>. If your cluster nodes use OS X, see the section, link:http://wiki.apache.org/hadoop/Running_Hadoop_On_OS_X_10.5_64-bit_%28Single-Node_Cluster%29[SSH: Setting up Remote Desktop and Enabling Self-Login] on the Hadoop wiki.
HBase uses the Secure Shell (ssh) command and utilities extensively to communicate between cluster nodes. Each server in the cluster must be running `ssh` so that the Hadoop and HBase daemons can be managed. You must be able to connect to all nodes via SSH, including the local node, from the Master as well as any backup Master, using a shared key rather than a password. You can see the basic methodology for such a set-up in Linux or Unix systems at "<<passwordless.ssh.quickstart>>". If your cluster nodes use OS X, see the section, link:http://wiki.apache.org/hadoop/Running_Hadoop_On_OS_X_10.5_64-bit_%28Single-Node_Cluster%29[SSH: Setting up Remote Desktop and Enabling Self-Login] on the Hadoop wiki.
DNS::
HBase uses the local hostname to self-report its IP address. Both forward and reverse DNS resolving must work in versions of HBase previous to 0.92.0. The link:https://github.com/sujee/hadoop-dns-checker[hadoop-dns-checker] tool can be used to verify DNS is working correctly on the cluster. The project README file provides detailed instructions on usage.
HBase uses the local hostname to self-report its IP address. Both forward and reverse DNS resolving must work in versions of HBase previous to 0.92.0. The link:https://github.com/sujee/hadoop-dns-checker[hadoop-dns-checker] tool can be used to verify DNS is working correctly on the cluster. The project `README` file provides detailed instructions on usage.
Loopback IP::
Prior to hbase-0.96.0, HBase only used the IP address [systemitem]+127.0.0.1+ to refer to `localhost`, and this could not be configured.
See <<loopback.ip,loopback.ip>>.
Prior to hbase-0.96.0, HBase only used the IP address `127.0.0.1` to refer to `localhost`, and this could not be configured.
See <<loopback.ip,Loopback IP>> for more details.
NTP::
The clocks on cluster nodes should be synchronized. A small amount of variation is acceptable, but larger amounts of skew can cause erratic and unexpected behavior. Time synchronization is one of the first things to check if you see unexplained problems in your cluster. It is recommended that you run a Network Time Protocol (NTP) service, or another time-synchronization mechanism, on your cluster, and that all nodes look to the same service for time synchronization. See the link:http://www.tldp.org/LDP/sag/html/basic-ntp-config.html[Basic NTP Configuration] at [citetitle]_The Linux Documentation Project (TLDP)_ to set up NTP.
Limits on Number of Files and Processes (ulimit)::
Apache HBase is a database. It requires the ability to open a large number of files at once. Many Linux distributions limit the number of files a single user is allowed to open to `1024` (or `256` on older versions of OS X). You can check this limit on your servers by running the command +ulimit -n+ when logged in as the user which runs HBase. See <<trouble.rs.runtime.filehandles,trouble.rs.runtime.filehandles>> for some of the problems you may experience if the limit is too low. You may also notice errors such as the following:
Apache HBase is a database. It requires the ability to open a large number of files at once. Many Linux distributions limit the number of files a single user is allowed to open to `1024` (or `256` on older versions of OS X). You can check this limit on your servers by running the command `ulimit -n` when logged in as the user which runs HBase. See <<trouble.rs.runtime.filehandles,the Troubleshooting section>> for some of the problems you may experience if the limit is too low. You may also notice errors such as the following:
+
----
2010-04-06 03:04:37,542 INFO org.apache.hadoop.hdfs.DFSClient: Exception increateBlockOutputStream java.io.EOFException
2010-04-06 03:04:37,542 INFO org.apache.hadoop.hdfs.DFSClient: Abandoning block blk_-6935524980745310745_1391901
----
+
It is recommended to raise the ulimit to at least 10,000, but more likely 10,240, because the value is usually expressed in multiples of 1024. Each ColumnFamily has at least one StoreFile, and possibly more than 6 StoreFiles if the region is under load. The number of open files required depends upon the number of ColumnFamilies and the number of regions. The following is a rough formula for calculating the potential number of open files on a RegionServer.
It is recommended to raise the ulimit to at least 10,000, but more likely 10,240, because the value is usually expressed in multiples of 1024. Each ColumnFamily has at least one StoreFile, and possibly more than six StoreFiles if the region is under load. The number of open files required depends upon the number of ColumnFamilies and the number of regions. The following is a rough formula for calculating the potential number of open files on a RegionServer.
+
.Calculate the Potential Number of Open Files
----
@ -156,18 +155,18 @@ It is recommended to raise the ulimit to at least 10,000, but more likely 10,240
+
For example, assuming that a schema had 3 ColumnFamilies per region with an average of 3 StoreFiles per ColumnFamily, and there are 100 regions per RegionServer, the JVM will open `3 * 3 * 100 = 900` file descriptors, not counting open JAR files, configuration files, and others. Opening a file does not take many resources, and the risk of allowing a user to open too many files is minimal.
+
Another related setting is the number of processes a user is allowed to run at once. In Linux and Unix, the number of processes is set using the ulimit -u command. This should not be confused with the nproc command, which controls the number of CPUs available to a given user. Under load, a nproc that is too low can cause OutOfMemoryError exceptions. See Jack Levin's major hdfs issues thread on the hbase-users mailing list, from 2011.
Another related setting is the number of processes a user is allowed to run at once. In Linux and Unix, the number of processes is set using the `ulimit -u` command. This should not be confused with the `nproc` command, which controls the number of CPUs available to a given user. Under load, a `ulimit -u` that is too low can cause OutOfMemoryError exceptions. See Jack Levin's major HDFS issues thread on the hbase-users mailing list, from 2011.
+
Configuring the maximum number of ile descriptors and processes for the user who is running the HBase process is an operating system configuration, rather than an HBase configuration. It is also important to be sure that the settings are changed for the user that actually runs HBase. To see which user started HBase, and that user's ulimit configuration, look at the first line of the HBase log for that instance. A useful read setting config on you hadoop cluster is Aaron Kimballs' Configuration Parameters: What can you just ignore?
+
.`ulimit` Settings on Ubuntu
Configuring the maximum number of file descriptors and processes for the user who is running the HBase process is an operating system configuration, rather than an HBase configuration. It is also important to be sure that the settings are changed for the user that actually runs HBase. To see which user started HBase, and that user's ulimit configuration, look at the first line of the HBase log for that instance. A useful read setting config on you hadoop cluster is Aaron Kimballs' Configuration Parameters: What can you just ignore?
+
.`ulimit` Settings on Ubuntu
====
To configure ulimit settings on Ubuntu, edit /etc/security/limits.conf, which is a space-delimited file with four columns. Refer to the man page for limits.conf for details about the format of this file. In the following example, the first line sets both soft and hard limits for the number of open files (nofile) to 32768 for the operating system user with the username hadoop. The second line sets the number of processes to 32000 for the same user.
To configure ulimit settings on Ubuntu, edit _/etc/security/limits.conf_, which is a space-delimited file with four columns. Refer to the man page for _limits.conf_ for details about the format of this file. In the following example, the first line sets both soft and hard limits for the number of open files (nofile) to 32768 for the operating system user with the username hadoop. The second line sets the number of processes to 32000 for the same user.
----
hadoop - nofile 32768
hadoop - nproc 32000
----
The settings are only applied if the Pluggable Authentication Module (PAM) environment is directed to use them. To configure PAM to use these limits, be sure that the /etc/pam.d/common-session file contains the following line:
The settings are only applied if the Pluggable Authentication Module (PAM) environment is directed to use them. To configure PAM to use these limits, be sure that the _/etc/pam.d/common-session_ file contains the following line:
----
session required pam_limits.so
----
@ -185,7 +184,7 @@ The following table summarizes the versions of Hadoop supported with each versio
Based on the version of HBase, you should select the most appropriate version of Hadoop.
You can use Apache Hadoop, or a vendor's distribution of Hadoop.
No distinction is made here.
See link:http://wiki.apache.org/hadoop/Distributions%20and%20Commercial%20Support for information about vendors of Hadoop.
See link:http://wiki.apache.org/hadoop/Distributions%20and%20Commercial%20Support[the Hadoop wiki] for information about vendors of Hadoop.
.Hadoop 2.x is recommended.
[TIP]
@ -198,9 +197,14 @@ HBase 0.98 drops support for Hadoop 1.0, deprecates use of Hadoop 1.1+, and HBas
Use the following legend to interpret this table:
.Hadoop version support matrix
* "S" = supported
* "X" = not supported
* "NT" = Not tested
[cols="1,1,1,1,1,1", options="header"]
|===
| | HBase-0.92.x | HBase-0.94.x | HBase-0.96.x | HBase-0.98.x (Support for Hadoop 1.1+ is deprecated.) | HBase-1.0.x (Hadoop 1.x is NOT supported)
| | HBase-0.92.x | HBase-0.94.x | HBase-0.96.x | HBase-0.98.x (Support for Hadoop 1.1+ is deprecated.) | HBase-1.0.x (Hadoop 1.x is NOT supported)
|Hadoop-0.20.205 | S | X | X | X | X
|Hadoop-0.22.x | S | X | X | X | X
|Hadoop-1.0.x |X | X | X | X | X
@ -222,13 +226,13 @@ The bundled jar is ONLY for use in standalone mode.
In distributed mode, it is _critical_ that the version of Hadoop that is out on your cluster match what is under HBase.
Replace the hadoop jar found in the HBase lib directory with the hadoop jar you are running on your cluster to avoid version mismatch issues.
Make sure you replace the jar in HBase everywhere on your cluster.
Hadoop version mismatch issues have various manifestations but often all looks like its hung up.
Hadoop version mismatch issues have various manifestations but often all looks like its hung up.
====
[[hadoop2.hbase_0.94]]
==== Apache HBase 0.94 with Hadoop 2
To get 0.94.x to run on hadoop 2.2.0, you need to change the hadoop 2 and protobuf versions in the _pom.xml_: Here is a diff with pom.xml changes:
To get 0.94.x to run on Hadoop 2.2.0, you need to change the hadoop 2 and protobuf versions in the _pom.xml_: Here is a diff with pom.xml changes:
[source]
----
@ -259,23 +263,23 @@ Index: pom.xml
The next step is to regenerate Protobuf files and assuming that the Protobuf has been installed:
* Go to the hbase root folder, using the command line;
* Go to the HBase root folder, using the command line;
* Type the following commands:
+
[source,bourne]
----
$ protoc -Isrc/main/protobuf --java_out=src/main/java src/main/protobuf/hbase.proto
----
----
+
[source,bourne]
----
$ protoc -Isrc/main/protobuf --java_out=src/main/java src/main/protobuf/ErrorHandling.proto
----
----
Building against the hadoop 2 profile by running something like the following command:
Building against the hadoop 2 profile by running something like the following command:
----
$ mvn clean install assembly:single -Dhadoop.profile=2.0 -DskipTests
@ -292,7 +296,7 @@ HBase-0.94 can additionally work with Hadoop-0.23.x and 2.x, but you may have to
As of Apache HBase 0.96.x, Apache Hadoop 1.0.x at least is required.
Hadoop 2 is strongly encouraged (faster but also has fixes that help MTTR). We will no longer run properly on older Hadoops such as 0.20.205 or branch-0.20-append.
Do not move to Apache HBase 0.96.x if you cannot upgrade your Hadoop.. See link:http://search-hadoop.com/m/7vFVx4EsUb2[HBase, mail # dev - DISCUSS:
Do not move to Apache HBase 0.96.x if you cannot upgrade your Hadoop. See link:http://search-hadoop.com/m/7vFVx4EsUb2[HBase, mail # dev - DISCUSS:
Have hbase require at least hadoop 1.0.0 in hbase 0.96.0?]
[[hadoop.older.versions]]
@ -303,13 +307,13 @@ DO NOT use Hadoop 0.20.2, Hadoop 0.20.203.0, and Hadoop 0.20.204.0 which DO NOT
Currently only Hadoop versions 0.20.205.x or any release in excess of this version -- this includes hadoop-1.0.0 -- have a working, durable sync.
The Cloudera blog post link:http://www.cloudera.com/blog/2012/01/an-update-on-apache-hadoop-1-0/[An
update on Apache Hadoop 1.0] by Charles Zedlweski has a nice exposition on how all the Hadoop versions relate.
Its worth checking out if you are having trouble making sense of the Hadoop version morass.
It's worth checking out if you are having trouble making sense of the Hadoop version morass.
Sync has to be explicitly enabled by setting `dfs.support.append` equal to true on both the client side -- in _hbase-site.xml_ -- and on the serverside in _hdfs-site.xml_ (The sync facility HBase needs is a subset of the append code path).
[source,xml]
----
<property>
<name>dfs.support.append</name>
<value>true</value>
@ -317,7 +321,7 @@ Sync has to be explicitly enabled by setting `dfs.support.append` equal to true
----
You will have to restart your cluster after making this edit.
Ignore the chicken-little comment you'll find in the _hdfs-default.xml_ in the description for the `dfs.support.append` configuration.
Ignore the chicken-little comment you'll find in the _hdfs-default.xml_ in the description for the `dfs.support.append` configuration.
[[hadoop.security]]
==== Apache HBase on Secure Hadoop
@ -325,12 +329,12 @@ Ignore the chicken-little comment you'll find in the _hdfs-default.xml_ in the d
Apache HBase will run on any Hadoop 0.20.x that incorporates Hadoop security features as long as you do as suggested above and replace the Hadoop jar that ships with HBase with the secure version.
If you want to read more about how to setup Secure HBase, see <<hbase.secure.configuration,hbase.secure.configuration>>.
`dfs.datanode.max.transfer.threads`
[[dfs.datanode.max.transfer.threads]]
==== (((dfs.datanode.max.transfer.threads)))
An HDFS datanode has an upper bound on the number of files that it will serve at any one time.
Before doing any loading, make sure you have configured Hadoop's _conf/hdfs-site.xml_, setting the `dfs.datanode.max.transfer.threads` value to at least the following:
[[dfs.datanode.max.transfer.threads]]
==== `dfs.datanode.max.transfer.threads` (((dfs.datanode.max.transfer.threads)))
An HDFS DataNode has an upper bound on the number of files that it will serve at any one time.
Before doing any loading, make sure you have configured Hadoop's _conf/hdfs-site.xml_, setting the `dfs.datanode.max.transfer.threads` value to at least the following:
[source,xml]
----
@ -353,24 +357,24 @@ For example:
contain current block. Will get new block locations from namenode and retry...
----
See also <<casestudies.max.transfer.threads,casestudies.max.transfer.threads>> and note that this property was previously known as `dfs.datanode.max.xcievers` (e.g. link:http://ccgtech.blogspot.com/2010/02/hadoop-hdfs-deceived-by-xciever.html[
Hadoop HDFS: Deceived by Xciever]).
See also <<casestudies.max.transfer.threads,casestudies.max.transfer.threads>> and note that this property was previously known as `dfs.datanode.max.xcievers` (e.g. link:http://ccgtech.blogspot.com/2010/02/hadoop-hdfs-deceived-by-xciever.html[Hadoop HDFS: Deceived by Xciever]).
[[zookeeper.requirements]]
=== ZooKeeper Requirements
ZooKeeper 3.4.x is required as of HBase 1.0.0.
HBase makes use of the [method]+multi+ functionality that is only available since 3.4.0 (The +useMulti+ is defaulted true in HBase 1.0.0). See link:[HBASE-12241 The crash of regionServer when taking deadserver's replication queue breaks replication] and link:[Use ZK.multi when available for HBASE-6710 0.92/0.94 compatibility fix] for background.
HBase makes use of the `multi` functionality that is only available since 3.4.0 (The `useMulti` configuration option defaults to `true` in HBase 1.0.0).
See link:https://issues.apache.org/jira/browse/HBASE-12241[HBASE-12241 (The crash of regionServer when taking deadserver's replication queue breaks replication)] and link:https://issues.apache.org/jira/browse/HBASE-6775[HBASE-6775 (Use ZK.multi when available for HBASE-6710 0.92/0.94 compatibility fix)] for background.
[[standalone_dist]]
== HBase run modes: Standalone and Distributed
HBase has two run modes: <<standalone,standalone>> and <<distributed,distributed>>.
Out of the box, HBase runs in standalone mode.
Whatever your mode, you will need to configure HBase by editing files in the HBase _conf_ directory.
At a minimum, you must edit `conf/hbase-env.sh` to tell HBase which +java+ to use.
In this file you set HBase environment variables such as the heapsize and other options for the +JVM+, the preferred location for log files, etc.
Set `JAVA_HOME` to point at the root of your +java+ install.
Whatever your mode, you will need to configure HBase by editing files in the HBase _conf_ directory.
At a minimum, you must edit [code]+conf/hbase-env.sh+ to tell HBase which +java+ to use.
In this file you set HBase environment variables such as the heapsize and other options for the `JVM`, the preferred location for log files, etc.
Set [var]+JAVA_HOME+ to point at the root of your +java+ install.
[[standalone]]
=== Standalone HBase
@ -382,17 +386,12 @@ Zookeeper binds to a well known port so clients may talk to HBase.
=== Distributed
Distributed mode can be subdivided into distributed but all daemons run on a single node -- a.k.a _pseudo-distributed_-- and _fully-distributed_ where the daemons are spread across all nodes in the cluster.
The pseudo-distributed vs fully-distributed nomenclature comes from Hadoop.
Distributed mode can be subdivided into distributed but all daemons run on a single node -- a.k.a _pseudo-distributed_ -- and _fully-distributed_ where the daemons are spread across all nodes in the cluster.
The _pseudo-distributed_ vs. _fully-distributed_ nomenclature comes from Hadoop.
Pseudo-distributed mode can run against the local filesystem or it can run against an instance of the _Hadoop Distributed File System_ (HDFS). Fully-distributed mode can ONLY run on HDFS.
See the Hadoop link:http://hadoop.apache.org/common/docs/r1.1.1/api/overview-summary.html#overview_description[
requirements and instructions] for how to set up HDFS for Hadoop 1.x.
A good walk-through for setting up HDFS on Hadoop 2 is at link:http://www.alexjf.net/blog/distributed-systems/hadoop-yarn-installation-definitive-guide.
Below we describe the different distributed setups.
Starting, verification and exploration of your install, whether a _pseudo-distributed_ or _fully-distributed_ configuration is described in a section that follows, <<confirm,confirm>>.
The same verification script applies to both deploy types.
See the Hadoop link:http://hadoop.apache.org/docs/current/[documentation] for how to set up HDFS.
A good walk-through for setting up HDFS on Hadoop 2 can be found at http://www.alexjf.net/blog/distributed-systems/hadoop-yarn-installation-definitive-guide.
[[pseudo]]
==== Pseudo-distributed
@ -418,7 +417,7 @@ For a production environment, distributed mode is appropriate.
In distributed mode, multiple instances of HBase daemons run on multiple servers in the cluster.
Just as in pseudo-distributed mode, a fully distributed configuration requires that you set the `hbase-cluster.distributed` property to `true`.
Typically, the `hbase.rootdir` is configured to point to a highly-available HDFS filesystem.
Typically, the `hbase.rootdir` is configured to point to a highly-available HDFS filesystem.
In addition, the cluster is configured so that multiple cluster nodes enlist as RegionServers, ZooKeeper QuorumPeers, and backup HMaster servers.
These configuration basics are all demonstrated in <<quickstart_fully_distributed,quickstart-fully-distributed>>.
@ -430,14 +429,14 @@ Each host is on a separate line.
All hosts listed in this file will have their RegionServer processes started and stopped when the master server starts or stops.
.ZooKeeper and HBase
See section <<zookeeper,zookeeper>> for ZooKeeper setup for HBase.
See the <<zookeeper,ZooKeeper>> section for ZooKeeper setup instructions for HBase.
.Example Distributed HBase Cluster
====
This is a bare-bones _conf/hbase-site.xml_ for a distributed HBase cluster.
A cluster that is used for real-world work would contain more custom configuration parameters.
Most HBase configuration directives have default values, which are used unless the value is overridden in the _hbase-site.xml_.
See <<config.files,config.files>> for more information.
See "<<config.files,Configuration Files>>" for more information.
[source,xml]
----
@ -452,14 +451,14 @@ See <<config.files,config.files>> for more information.
<value>true</value>
</property>
<property>
<name>hbase.zookeeper.quorum</name>
<value>node-a.example.com,node-b.example.com,node-c.example.com</value>
</property>
<name>hbase.zookeeper.quorum</name>
<value>node-a.example.com,node-b.example.com,node-c.example.com</value>
</property>
</configuration>
----
This is an example _conf/regionservers_ file, which contains a list of each node that should run a RegionServer in the cluster.
These nodes need HBase installed and they need to use the same contents of the _conf/_ directory as the Master server..
This is an example _conf/regionservers_ file, which contains a list of nodes that should run a RegionServer in the cluster.
These nodes need HBase installed and they need to use the same contents of the _conf/_ directory as the Master server
[source]
----
@ -484,7 +483,7 @@ node-c.example.com
See <<quickstart_fully_distributed,quickstart-fully-distributed>> for a walk-through of a simple three-node cluster configuration with multiple ZooKeeper, backup HMaster, and RegionServer instances.
.Procedure: HDFS Client Configuration
. Of note, if you have made HDFS client configuration on your Hadoop cluster, such as configuration directives for HDFS clients, as opposed to server-side configurations, you must use one of the following methods to enable HBase to see and use these configuration changes:
. Of note, if you have made HDFS client configuration changes on your Hadoop cluster, such as configuration directives for HDFS clients, as opposed to server-side configurations, you must use one of the following methods to enable HBase to see and use these configuration changes:
+
a. Add a pointer to your `HADOOP_CONF_DIR` to the `HBASE_CLASSPATH` environment variable in _hbase-env.sh_.
b. Add a copy of _hdfs-site.xml_ (or _hadoop-site.xml_) or, better, symlinks, under _${HBASE_HOME}/conf_, or
@ -492,18 +491,17 @@ c. if only a small set of HDFS client configurations, add them to _hbase-site.xm
An example of such an HDFS client configuration is `dfs.replication`.
If for example, you want to run with a replication factor of 5, hbase will create files with the default of 3 unless you do the above to make the configuration available to HBase.
If for example, you want to run with a replication factor of 5, HBase will create files with the default of 3 unless you do the above to make the configuration available to HBase.
[[confirm]]
== Running and Confirming Your Installation
Make sure HDFS is running first.
Start and stop the Hadoop HDFS daemons by running _bin/start-hdfs.sh_ over in the `HADOOP_HOME` directory.
You can ensure it started properly by testing the +put+ and +get+ of files into the Hadoop filesystem.
HBase does not normally use the mapreduce daemons.
These do not need to be started.
Start and stop the Hadoop HDFS daemons by running _bin/start-hdfs.sh_ over in the `HADOOP_HOME` directory.
You can ensure it started properly by testing the `put` and `get` of files into the Hadoop filesystem.
HBase does not normally use the MapReduce or YARN daemons. These do not need to be started.
_If_ you are managing your own ZooKeeper, start it and confirm its running else, HBase will start up ZooKeeper for you as part of its start process.
_If_ you are managing your own ZooKeeper, start it and confirm it's running, else HBase will start up ZooKeeper for you as part of its start process.
Start HBase with the following command:
@ -518,11 +516,11 @@ HBase logs can be found in the _logs_ subdirectory.
Check them out especially if HBase had trouble starting.
HBase also puts up a UI listing vital attributes.
By default its deployed on the Master host at port 16010 (HBase RegionServers listen on port 16020 by default and put up an informational http server at 16030). If the Master were running on a host named `master.example.org` on the default port, to see the Master's homepage you'd point your browser at _http://master.example.org:16010_.
By default it's deployed on the Master host at port 16010 (HBase RegionServers listen on port 16020 by default and put up an informational HTTP server at port 16030). If the Master is running on a host named `master.example.org` on the default port, point your browser at _http://master.example.org:16010_ to see the web interface.
Prior to HBase 0.98, the default ports the master ui was deployed on port 16010, and the HBase RegionServers would listen on port 16020 by default and put up an informational http server at 16030.
Prior to HBase 0.98 the master UI was deployed on port 60010, and the HBase RegionServers UI on port 60030.
Once HBase has started, see the <<shell_exercises,shell exercises>> for how to create tables, add data, scan your insertions, and finally disable and drop your tables.
Once HBase has started, see the <<shell_exercises,shell exercises>> section for how to create tables, add data, scan your insertions, and finally disable and drop your tables.
To stop HBase after exiting the HBase shell enter
@ -545,11 +543,11 @@ Just as in Hadoop where you add site-specific HDFS configuration to the _hdfs-si
For the list of configurable properties, see <<hbase_default_configurations,hbase default configurations>> below or view the raw _hbase-default.xml_ source file in the HBase source code at _src/main/resources_.
Not all configuration options make it out to _hbase-default.xml_.
Configuration that it is thought rare anyone would change can exist only in code; the only way to turn up such configurations is via a reading of the source code itself.
Configuration that it is thought rare anyone would change can exist only in code; the only way to turn up such configurations is via a reading of the source code itself.
Currently, changes here will require a cluster restart for HBase to notice the change.
Currently, changes here will require a cluster restart for HBase to notice the change.
// hbase/src/main/asciidoc
//
//
include::../../../../target/asciidoc/hbase-default.adoc[]
@ -563,14 +561,14 @@ Open the file at _conf/hbase-env.sh_ and peruse its content.
Each option is fairly well documented.
Add your own environment variables here if you want them read by HBase daemons on startup.
Changes here will require a cluster restart for HBase to notice the change.
Changes here will require a cluster restart for HBase to notice the change.
[[log4j]]
=== _log4j.properties_
Edit this file to change rate at which HBase files are rolled and to change the level at which HBase logs messages.
Edit this file to change rate at which HBase files are rolled and to change the level at which HBase logs messages.
Changes here will require a cluster restart for HBase to notice the change though log levels can be changed for particular daemons via the HBase UI.
Changes here will require a cluster restart for HBase to notice the change though log levels can be changed for particular daemons via the HBase UI.
[[client_dependencies]]
=== Client configuration and dependencies connecting to an HBase cluster
@ -579,12 +577,12 @@ If you are running HBase in standalone mode, you don't need to configure anythin
Since the HBase Master may move around, clients bootstrap by looking to ZooKeeper for current critical locations.
ZooKeeper is where all these values are kept.
Thus clients require the location of the ZooKeeper ensemble information before they can do anything else.
Usually this the ensemble location is kept out in the _hbase-site.xml_ and is picked up by the client from the `CLASSPATH`.
Thus clients require the location of the ZooKeeper ensemble before they can do anything else.
Usually this the ensemble location is kept out in the _hbase-site.xml_ and is picked up by the client from the `CLASSPATH`.
If you are configuring an IDE to run a HBase client, you should include the _conf/_ directory on your classpath so _hbase-site.xml_ settings can be found (or add _src/test/resources_ to pick up the hbase-site.xml used by tests).
Minimally, a client of HBase needs several libraries in its `CLASSPATH` when connecting to a cluster, including:
Minimally, a client of HBase needs several libraries in its `CLASSPATH` when connecting to a cluster, including:
[source]
----
@ -597,7 +595,7 @@ log4j (log4j-1.2.16.jar)
slf4j-api (slf4j-api-1.5.8.jar)
slf4j-log4j (slf4j-log4j12-1.5.8.jar)
zookeeper (zookeeper-3.4.2.jar)
----
----
An example basic _hbase-site.xml_ for client only might look as follows:
[source,xml]
@ -612,36 +610,37 @@ An example basic _hbase-site.xml_ for client only might look as follows:
</description>
</property>
</configuration>
----
----
[[java.client.config]]
==== Java client configuration
The configuration used by a Java client is kept in an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HBaseConfiguration[HBaseConfiguration] instance.
The configuration used by a Java client is kept in an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HBaseConfiguration[HBaseConfiguration] instance.
The factory method on HBaseConfiguration, `HBaseConfiguration.create();`, on invocation, will read in the content of the first _hbase-site.xml_ found on the client's `CLASSPATH`, if one is present (Invocation will also factor in any _hbase-default.xml_ found; an hbase-default.xml ships inside the _hbase.X.X.X.jar_). It is also possible to specify configuration directly without having to read from a _hbase-site.xml_.
For example, to set the ZooKeeper ensemble for the cluster programmatically do as follows:
The factory method on HBaseConfiguration, `HBaseConfiguration.create();`, on invocation, will read in the content of the first _hbase-site.xml_ found on the client's `CLASSPATH`, if one is present (Invocation will also factor in any _hbase-default.xml_ found; an _hbase-default.xml_ ships inside the _hbase.X.X.X.jar_). It is also possible to specify configuration directly without having to read from a _hbase-site.xml_.
For example, to set the ZooKeeper ensemble for the cluster programmatically do as follows:
[source,java]
----
Configuration config = HBaseConfiguration.create();
config.set("hbase.zookeeper.quorum", "localhost"); // Here we are running zookeeper locally
----
----
If multiple ZooKeeper instances make up your ZooKeeper ensemble, they may be specified in a comma-separated list (just as in the _hbase-site.xml_ file). This populated `Configuration` instance can then be passed to an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html[HTable], and so on.
If multiple ZooKeeper instances make up your ZooKeeper ensemble, they may be specified in a comma-separated list (just as in the _hbase-site.xml_ file). This populated `Configuration` instance can then be passed to an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html[HTable], and so on.
[[example_config]]
== Example Configurations
=== Basic Distributed HBase Install
Here is an example basic configuration for a distributed ten node cluster.
The nodes are named `example0`, `example1`, etc., through node `example9` in this example.
The HBase Master and the HDFS namenode are running on the node `example0`.
RegionServers run on nodes `example1`-`example9`.
A 3-node ZooKeeper ensemble runs on `example1`, `example2`, and `example3` on the default ports.
ZooKeeper data is persisted to the directory _/export/zookeeper_.
Below we show what the main configuration files -- _hbase-site.xml_, _regionservers_, and _hbase-env.sh_ -- found in the HBase _conf_ directory might look like.
Here is an example basic configuration for a distributed ten node cluster:
* The nodes are named `example0`, `example1`, etc., through node `example9` in this example.
* The HBase Master and the HDFS NameNode are running on the node `example0`.
* RegionServers run on nodes `example1`-`example9`.
* A 3-node ZooKeeper ensemble runs on `example1`, `example2`, and `example3` on the default ports.
* ZooKeeper data is persisted to the directory _/export/zookeeper_.
Below we show what the main configuration files -- _hbase-site.xml_, _regionservers_, and _hbase-env.sh_ -- found in the HBase _conf_ directory might look like.
[[hbase_site]]
==== _hbase-site.xml_
@ -685,7 +684,7 @@ Below we show what the main configuration files -- _hbase-site.xml_, _regionserv
==== _regionservers_
In this file you list the nodes that will run RegionServers.
In our case, these nodes are `example1`-`example9`.
In our case, these nodes are `example1`-`example9`.
[source]
----
@ -705,46 +704,43 @@ example9
The following lines in the _hbase-env.sh_ file show how to set the `JAVA_HOME` environment variable (required for HBase 0.98.5 and newer) and set the heap to 4 GB (rather than the default value of 1 GB). If you copy and paste this example, be sure to adjust the `JAVA_HOME` to suit your environment.
[source,bash]
----
# The java implementation to use.
export JAVA_HOME=/usr/java/jdk1.7.0/
export JAVA_HOME=/usr/java/jdk1.7.0/
# The maximum amount of heap to use, in MB. Default is 1000.
export HBASE_HEAPSIZE=4096
----
Use +rsync+ to copy the content of the _conf_ directory to all nodes of the cluster.
Use +rsync+ to copy the content of the _conf_ directory to all nodes of the cluster.
[[important_configurations]]
== The Important Configurations
Below we list what the _important_ Configurations.
We've divided this section into required configuration and worth-a-look recommended configs.
Below we list some _important_ configurations.
We've divided this section into required configuration and worth-a-look recommended configs.
[[required_configuration]]
=== Required Configurations
Review the <<os,os>> and <<hadoop,hadoop>> sections.
Review the <<os,os>> and <<hadoop,hadoop>> sections.
[[big.cluster.config]]
==== Big Cluster Configurations
If a cluster with a lot of regions, it is possible if an eager beaver regionserver checks in soon after master start while all the rest in the cluster are laggardly, this first server to checkin will be assigned all regions.
If lots of regions, this first server could buckle under the load.
To prevent the above scenario happening up the `hbase.master.wait.on.regionservers.mintostart` from its default value of 1.
If you have a cluster with a lot of regions, it is possible that a Regionserver checks in briefly after the Master starts while all the remaining RegionServers lag behind. This first server to check in will be assigned all regions which is not optimal.
To prevent the above scenario from happening, up the `hbase.master.wait.on.regionservers.mintostart` property from its default value of 1.
See link:https://issues.apache.org/jira/browse/HBASE-6389[HBASE-6389 Modify the
conditions to ensure that Master waits for sufficient number of Region Servers before
starting region assignments] for more detail.
starting region assignments] for more detail.
[[backup.master.fail.fast]]
==== If a backup Master, making primary Master fail fast
==== If a backup Master exists, make the primary Master fail fast
If the primary Master loses its connection with ZooKeeper, it will fall into a loop where it keeps trying to reconnect.
Disable this functionality if you are running more than one Master: i.e.
a backup Master.
Disable this functionality if you are running more than one Master: i.e. a backup Master.
Failing to do so, the dying Master may continue to receive RPCs though another Master has assumed the role of primary.
See the configuration <<fail.fast.expired.active.master,fail.fast.expired.active.master>>.
See the configuration <<fail.fast.expired.active.master,fail.fast.expired.active.master>>.
=== Recommended Configurations
@ -760,15 +756,15 @@ Before changing this value, be sure you have your JVM garbage collection configu
To change this configuration, edit _hbase-site.xml_, copy the changed file around the cluster and restart.
We set this value high to save our having to field noob questions up on the mailing lists asking why a RegionServer went down during a massive import.
We set this value high to save our having to field questions up on the mailing lists asking why a RegionServer went down during a massive import.
The usual cause is that their JVM is untuned and they are running into long GC pauses.
Our thinking is that while users are getting familiar with HBase, we'd save them having to know all of its intricacies.
Later when they've built some confidence, then they can play with configuration such as this.
Later when they've built some confidence, then they can play with configuration such as this.
[[zookeeper.instances]]
===== Number of ZooKeeper Instances
See <<zookeeper,zookeeper>>.
See <<zookeeper,zookeeper>>.
[[recommended.configurations.hdfs]]
==== HDFS Configurations
@ -776,35 +772,35 @@ See <<zookeeper,zookeeper>>.
[[dfs.datanode.failed.volumes.tolerated]]
===== dfs.datanode.failed.volumes.tolerated
This is the "...number of volumes that are allowed to fail before a datanode stops offering service.
This is the "...number of volumes that are allowed to fail before a DataNode stops offering service.
By default any volume failure will cause a datanode to shutdown" from the _hdfs-default.xml_ description.
If you have > three or four disks, you might want to set this to 1 or if you have many disks, two or more.
You might want to set this to about half the amount of your available disks.
[[hbase.regionserver.handler.count_description]]
==== `hbase.regionserver.handler.count`
This setting defines the number of threads that are kept open to answer incoming requests to user tables.
The rule of thumb is to keep this number low when the payload per request approaches the MB (big puts, scans using a large cache) and high when the payload is small (gets, small puts, ICVs, deletes). The total size of the queries in progress is limited by the setting "hbase.ipc.server.max.callqueue.size".
The rule of thumb is to keep this number low when the payload per request approaches the MB (big puts, scans using a large cache) and high when the payload is small (gets, small puts, ICVs, deletes). The total size of the queries in progress is limited by the setting `hbase.ipc.server.max.callqueue.size`.
It is safe to set that number to the maximum number of incoming clients if their payload is small, the typical example being a cluster that serves a website since puts aren't typically buffered and most of the operations are gets.
It is safe to set that number to the maximum number of incoming clients if their payload is small, the typical example being a cluster that serves a website since puts aren't typically buffered and most of the operations are gets.
The reason why it is dangerous to keep this setting high is that the aggregate size of all the puts that are currently happening in a region server may impose too much pressure on its memory, or even trigger an OutOfMemoryError.
A region server running on low memory will trigger its JVM's garbage collector to run more frequently up to a point where GC pauses become noticeable (the reason being that all the memory used to keep all the requests' payloads cannot be trashed, no matter how hard the garbage collector tries). After some time, the overall cluster throughput is affected since every request that hits that region server will take longer, which exacerbates the problem even more.
A RegionServer running on low memory will trigger its JVM's garbage collector to run more frequently up to a point where GC pauses become noticeable (the reason being that all the memory used to keep all the requests' payloads cannot be trashed, no matter how hard the garbage collector tries). After some time, the overall cluster throughput is affected since every request that hits that RegionServer will take longer, which exacerbates the problem even more.
You can get a sense of whether you have too little or too many handlers by <<rpc.logging,rpc.logging>> on an individual RegionServer then tailing its logs (Queued requests consume memory).
You can get a sense of whether you have too little or too many handlers by <<rpc.logging,rpc.logging>> on an individual RegionServer then tailing its logs (Queued requests consume memory).
[[big_memory]]
==== Configuration for large memory machines
HBase ships with a reasonable, conservative configuration that will work on nearly all machine types that people might want to test with.
If you have larger machines -- HBase has 8G and larger heap -- you might the following configuration options helpful.
TODO.
TODO.
[[config.compression]]
==== Compression
You should consider enabling ColumnFamily compression.
There are several options that are near-frictionless and in most all cases boost performance by reducing the size of StoreFiles and thus reducing I/O.
There are several options that are near-frictionless and in most all cases boost performance by reducing the size of StoreFiles and thus reducing I/O.
See <<compression,compression>> for more information.
@ -812,11 +808,11 @@ See <<compression,compression>> for more information.
==== Configuring the size and number of WAL files
HBase uses <<wal,wal>> to recover the memstore data that has not been flushed to disk in case of an RS failure.
These WAL files should be configured to be slightly smaller than HDFS block (by default, HDFS block is 64Mb and WAL file is ~60Mb).
These WAL files should be configured to be slightly smaller than HDFS block (by default a HDFS block is 64Mb and a WAL file is ~60Mb).
HBase also has a limit on number of WAL files, designed to ensure there's never too much data that needs to be replayed during recovery.
HBase also has a limit on the number of WAL files, designed to ensure there's never too much data that needs to be replayed during recovery.
This limit needs to be set according to memstore configuration, so that all the necessary data would fit.
It is recommended to allocated enough WAL files to store at least that much data (when all memstores are close to full). For example, with 16Gb RS heap, default memstore settings (0.4), and default WAL file size (~60Mb), 16Gb*0.4/60, the starting point for WAL file count is ~109.
It is recommended to allocate enough WAL files to store at least that much data (when all memstores are close to full). For example, with 16Gb RS heap, default memstore settings (0.4), and default WAL file size (~60Mb), 16Gb*0.4/60, the starting point for WAL file count is ~109.
However, as all memstores are not expected to be full all the time, less WAL files can be allocated.
[[disable.splitting]]
@ -832,7 +828,7 @@ Instead of allowing HBase to split your regions automatically, you can choose to
This feature was added in HBase 0.90.0.
Manually managing splits works if you know your keyspace well, otherwise let HBase figure where to split for you.
Manual splitting can mitigate region creation and movement under load.
It also makes it so region boundaries are known and invariant (if you disable region splitting). If you use manual splits, it is easier doing staggered, time-based major compactions spread out your network IO load.
It also makes it so region boundaries are known and invariant (if you disable region splitting). If you use manual splits, it is easier doing staggered, time-based major compactions to spread out your network IO load.
.Disable Automatic Splitting
To disable automatic splitting, set `hbase.hregion.max.filesize` to a very large value, such as `100 GB` It is not recommended to set it to its absolute maximum value of `Long.MAX_VALUE`.
@ -871,8 +867,7 @@ See the entry for `hbase.hregion.majorcompaction` in the <<compaction.parameters
====
Major compactions are absolutely necessary for StoreFile clean-up.
Do not disable them altogether.
You can run major compactions manually via the HBase shell or via the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html#majorCompact%28java.lang.String%29[HBaseAdmin
API].
You can run major compactions manually via the HBase shell or via the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html#majorCompact%28java.lang.String%29[HBaseAdmin API].
====
For more information about compactions and the compaction file selection process, see <<compaction,compaction>>
@ -881,7 +876,7 @@ For more information about compactions and the compaction file selection process
==== Speculative Execution
Speculative Execution of MapReduce tasks is on by default, and for HBase clusters it is generally advised to turn off Speculative Execution at a system-level unless you need it for a specific case, where it can be configured per-job.
Set the properties `mapreduce.map.speculative` and `mapreduce.reduce.speculative` to false.
Set the properties `mapreduce.map.speculative` and `mapreduce.reduce.speculative` to false.
[[other_configuration]]
=== Other Configurations
@ -890,98 +885,97 @@ Set the properties `mapreduce.map.speculative` and `mapreduce.reduce.speculative
==== Balancer
The balancer is a periodic operation which is run on the master to redistribute regions on the cluster.
It is configured via `hbase.balancer.period` and defaults to 300000 (5 minutes).
It is configured via `hbase.balancer.period` and defaults to 300000 (5 minutes).
See <<master.processes.loadbalancer,master.processes.loadbalancer>> for more information on the LoadBalancer.
See <<master.processes.loadbalancer,master.processes.loadbalancer>> for more information on the LoadBalancer.
[[disabling.blockcache]]
==== Disabling Blockcache
Do not turn off block cache (You'd do it by setting `hbase.block.cache.size` to zero). Currently we do not do well if you do this because the regionserver will spend all its time loading hfile indices over and over again.
If your working set it such that block cache does you no good, at least size the block cache such that hfile indices will stay up in the cache (you can get a rough idea on the size you need by surveying regionserver UIs; you'll see index block size accounted near the top of the webpage).
Do not turn off block cache (You'd do it by setting `hbase.block.cache.size` to zero). Currently we do not do well if you do this because the RegionServer will spend all its time loading HFile indices over and over again.
If your working set it such that block cache does you no good, at least size the block cache such that HFile indices will stay up in the cache (you can get a rough idea on the size you need by surveying RegionServer UIs; you'll see index block size accounted near the top of the webpage).
[[nagles]]
==== link:http://en.wikipedia.org/wiki/Nagle's_algorithm[Nagle's] or the small package problem
If a big 40ms or so occasional delay is seen in operations against HBase, try the Nagles' setting.
For example, see the user mailing list thread, link:http://search-hadoop.com/m/pduLg2fydtE/Inconsistent+scan+performance+with+caching+set+&subj=Re+Inconsistent+scan+performance+with+caching+set+to+1[Inconsistent scan performance with caching set to 1] and the issue cited therein where setting notcpdelay improved scan speeds.
You might also see the graphs on the tail of link:https://issues.apache.org/jira/browse/HBASE-7008[HBASE-7008 Set scanner caching to a better default] where our Lars Hofhansl tries various data sizes w/ Nagle's on and off measuring the effect.
For example, see the user mailing list thread, link:http://search-hadoop.com/m/pduLg2fydtE/Inconsistent+scan+performance+with+caching+set+&subj=Re+Inconsistent+scan+performance+with+caching+set+to+1[Inconsistent scan performance with caching set to 1] and the issue cited therein where setting `notcpdelay` improved scan speeds.
You might also see the graphs on the tail of link:https://issues.apache.org/jira/browse/HBASE-7008[HBASE-7008 Set scanner caching to a better default] where our Lars Hofhansl tries various data sizes w/ Nagle's on and off measuring the effect.
[[mttr]]
==== Better Mean Time to Recover (MTTR)
This section is about configurations that will make servers come back faster after a fail.
See the Deveraj Das an Nicolas Liochon blog post link:http://hortonworks.com/blog/introduction-to-hbase-mean-time-to-recover-mttr/[Introduction to HBase Mean Time to Recover (MTTR)] for a brief introduction.
See the Deveraj Das an Nicolas Liochon blog post link:http://hortonworks.com/blog/introduction-to-hbase-mean-time-to-recover-mttr/[Introduction to HBase Mean Time to Recover (MTTR)] for a brief introduction.
The issue link:https://issues.apache.org/jira/browse/HBASE-8389[HBASE-8354 forces Namenode into loop with lease recovery requests] is messy but has a bunch of good discussion toward the end on low timeouts and how to effect faster recovery including citation of fixes added to HDFS.
Read the Varun Sharma comments.
The issue link:https://issues.apache.org/jira/browse/HBASE-8389[HBASE-8354 forces Namenode into loop with lease recovery requests] is messy but has a bunch of good discussion toward the end on low timeouts and how to effect faster recovery including citation of fixes added to HDFS. Read the Varun Sharma comments.
The below suggested configurations are Varun's suggestions distilled and tested.
Make sure you are running on a late-version HDFS so you have the fixes he refers too and himself adds to HDFS that help HBase MTTR (e.g.
HDFS-3703, HDFS-3712, and HDFS-4791 -- hadoop 2 for sure has them and late hadoop 1 has some). Set the following in the RegionServer.
HDFS-3703, HDFS-3712, and HDFS-4791 -- Hadoop 2 for sure has them and late Hadoop 1 has some). Set the following in the RegionServer.
[source,xml]
----
<property>
<name>hbase.lease.recovery.dfs.timeout</name>
<value>23000</value>
<description>How much time we allow elapse between calls to recover lease.
Should be larger than the dfs timeout.</description>
<name>hbase.lease.recovery.dfs.timeout</name>
<value>23000</value>
<description>How much time we allow elapse between calls to recover lease.
Should be larger than the dfs timeout.</description>
</property>
<property>
<name>dfs.client.socket-timeout</name>
<value>10000</value>
<description>Down the DFS timeout from 60 to 10 seconds.</description>
<name>dfs.client.socket-timeout</name>
<value>10000</value>
<description>Down the DFS timeout from 60 to 10 seconds.</description>
</property>
----
And on the namenode/datanode side, set the following to enable 'staleness' introduced in HDFS-3703, HDFS-3912.
And on the NameNode/DataNode side, set the following to enable 'staleness' introduced in HDFS-3703, HDFS-3912.
[source,xml]
----
<property>
<name>dfs.client.socket-timeout</name>
<value>10000</value>
<description>Down the DFS timeout from 60 to 10 seconds.</description>
<name>dfs.client.socket-timeout</name>
<value>10000</value>
<description>Down the DFS timeout from 60 to 10 seconds.</description>
</property>
<property>
<name>dfs.datanode.socket.write.timeout</name>
<value>10000</value>
<description>Down the DFS timeout from 8 * 60 to 10 seconds.</description>
<name>dfs.datanode.socket.write.timeout</name>
<value>10000</value>
<description>Down the DFS timeout from 8 * 60 to 10 seconds.</description>
</property>
<property>
<name>ipc.client.connect.timeout</name>
<value>3000</value>
<description>Down from 60 seconds to 3.</description>
<name>ipc.client.connect.timeout</name>
<value>3000</value>
<description>Down from 60 seconds to 3.</description>
</property>
<property>
<name>ipc.client.connect.max.retries.on.timeouts</name>
<value>2</value>
<description>Down from 45 seconds to 3 (2 == 3 retries).</description>
<name>ipc.client.connect.max.retries.on.timeouts</name>
<value>2</value>
<description>Down from 45 seconds to 3 (2 == 3 retries).</description>
</property>
<property>
<name>dfs.namenode.avoid.read.stale.datanode</name>
<value>true</value>
<description>Enable stale state in hdfs</description>
<name>dfs.namenode.avoid.read.stale.datanode</name>
<value>true</value>
<description>Enable stale state in hdfs</description>
</property>
<property>
<name>dfs.namenode.stale.datanode.interval</name>
<value>20000</value>
<description>Down from default 30 seconds</description>
<name>dfs.namenode.stale.datanode.interval</name>
<value>20000</value>
<description>Down from default 30 seconds</description>
</property>
<property>
<name>dfs.namenode.avoid.write.stale.datanode</name>
<value>true</value>
<description>Enable stale state in hdfs</description>
<name>dfs.namenode.avoid.write.stale.datanode</name>
<value>true</value>
<description>Enable stale state in hdfs</description>
</property>
----
[[jmx_config]]
==== JMX
JMX(Java Management Extensions) provides built-in instrumentation that enables you to monitor and manage the Java VM.
To enable monitoring and management from remote systems, you need to set system property com.sun.management.jmxremote.port(the port number through which you want to enable JMX RMI connections) when you start the Java VM.
See link:http://docs.oracle.com/javase/6/docs/technotes/guides/management/agent.html[official document] for more information.
Historically, besides above port mentioned, JMX opens 2 additional random TCP listening ports, which could lead to port conflict problem.(See link:https://issues.apache.org/jira/browse/HBASE-10289[HBASE-10289] for details)
JMX (Java Management Extensions) provides built-in instrumentation that enables you to monitor and manage the Java VM.
To enable monitoring and management from remote systems, you need to set system property `com.sun.management.jmxremote.port` (the port number through which you want to enable JMX RMI connections) when you start the Java VM.
See the link:http://docs.oracle.com/javase/6/docs/technotes/guides/management/agent.html[official documentation] for more information.
Historically, besides above port mentioned, JMX opens two additional random TCP listening ports, which could lead to port conflict problem. (See link:https://issues.apache.org/jira/browse/HBASE-10289[HBASE-10289] for details)
As an alternative, You can use the coprocessor-based JMX implementation provided by HBase.
To enable it in 0.99 or above, add below property in _hbase-site.xml_:
@ -989,31 +983,31 @@ To enable it in 0.99 or above, add below property in _hbase-site.xml_:
[source,xml]
----
<property>
<name>hbase.coprocessor.regionserver.classes</name>
<value>org.apache.hadoop.hbase.JMXListener</value>
<name>hbase.coprocessor.regionserver.classes</name>
<value>org.apache.hadoop.hbase.JMXListener</value>
</property>
----
----
NOTE: DO NOT set com.sun.management.jmxremote.port for Java VM at the same time.
NOTE: DO NOT set `com.sun.management.jmxremote.port` for Java VM at the same time.
Currently it supports Master and RegionServer Java VM.
The reason why you only configure coprocessor for 'regionserver' is that, starting from HBase 0.99, a Master IS also a RegionServer.
(See link:https://issues.apache.org/jira/browse/HBASE-10569[HBASE-10569] for more information.) By default, the JMX listens on TCP port 10102, you can further configure the port using below properties:
(See link:https://issues.apache.org/jira/browse/HBASE-10569[HBASE-10569] for more information.) By default, the JMX listens on TCP port 10102, you can further configure the port using below properties:
[source,xml]
----
<property>
<name>regionserver.rmi.registry.port</name>
<value>61130</value>
<name>regionserver.rmi.registry.port</name>
<value>61130</value>
</property>
<property>
<name>regionserver.rmi.connector.port</name>
<value>61140</value>
<name>regionserver.rmi.connector.port</name>
<value>61140</value>
</property>
----
----
The registry port can be shared with connector port in most cases, so you only need to configure regionserver.rmi.registry.port.
However if you want to use SSL communication, the 2 ports must be configured to different values.
However if you want to use SSL communication, the 2 ports must be configured to different values.
By default the password authentication and SSL communication is disabled.
To enable password authentication, you need to update _hbase-env.sh_ like below:
@ -1025,11 +1019,11 @@ export HBASE_JMX_BASE="-Dcom.sun.management.jmxremote.authenticate=true
export HBASE_MASTER_OPTS="$HBASE_MASTER_OPTS $HBASE_JMX_BASE "
export HBASE_REGIONSERVER_OPTS="$HBASE_REGIONSERVER_OPTS $HBASE_JMX_BASE "
----
----
See example password/access file under $JRE_HOME/lib/management.
See example password/access file under _$JRE_HOME/lib/management_.
To enable SSL communication with password authentication, follow below steps:
To enable SSL communication with password authentication, follow below steps:
[source,bash]
----
@ -1041,7 +1035,7 @@ keytool -export -alias jconsole -keystore myKeyStore -file jconsole.cert
#3. copy jconsole.cert to jconsole client machine, import it to jconsoleKeyStore
keytool -import -alias jconsole -keystore jconsoleKeyStore -file jconsole.cert
----
----
And then update _hbase-env.sh_ like below:
@ -1056,36 +1050,36 @@ export HBASE_JMX_BASE="-Dcom.sun.management.jmxremote.ssl=true
export HBASE_MASTER_OPTS="$HBASE_MASTER_OPTS $HBASE_JMX_BASE "
export HBASE_REGIONSERVER_OPTS="$HBASE_REGIONSERVER_OPTS $HBASE_JMX_BASE "
----
----
Finally start jconsole on client using the key store:
Finally start `jconsole` on the client using the key store:
[source,bash]
----
jconsole -J-Djavax.net.ssl.trustStore=/home/tianq/jconsoleKeyStore
----
----
NOTE: for HBase 0.98, To enable the HBase JMX implementation on Master, you also need to add below property in _hbase-site.xml_:
[source,xml]
----
<property>
<name>hbase.coprocessor.master.classes</name>
<value>org.apache.hadoop.hbase.JMXListener</value>
<ame>hbase.coprocessor.master.classes</name>
<value>org.apache.hadoop.hbase.JMXListener</value>
</property>
----
----
The corresponding properties for port configuration are master.rmi.registry.port (by default 10101) and master.rmi.connector.port(by default the same as registry.port)
The corresponding properties for port configuration are `master.rmi.registry.port` (by default 10101) and `master.rmi.connector.port` (by default the same as registry.port)
[[dyn_config]]
== Dynamic Configuration
Since HBase 1.0.0, it is possible to change a subset of the configuration without requiring a server restart.
In the hbase shell, there are new operators, +update_config+ and +update_all_config+ that will prompt a server or all servers to reload configuration.
In the HBase shell, there are new operators, `update_config` and `update_all_config` that will prompt a server or all servers to reload configuration.
Only a subset of all configurations can currently be changed in the running server.
Here is an incomplete list: +hbase.regionserver.thread.compaction.large+, +hbase.regionserver.thread.compaction.small+, +hbase.regionserver.thread.split+, +hbase.regionserver.thread.merge+, as well as compaction policy and configurations and adjustment to offpeak hours.
For the full list consult the patch attached to link:https://issues.apache.org/jira/browse/HBASE-12147[HBASE-12147 Porting Online Config Change from 89-fb].
Here is an incomplete list: `hbase.regionserver.thread.compaction.large`, `hbase.regionserver.thread.compaction.small`, `hbase.regionserver.thread.split`, `hbase.regionserver.thread.merge`, as well as compaction policy and configurations and adjustment to offpeak hours.
For the full list consult the patch attached to link:https://issues.apache.org/jira/browse/HBASE-12147[HBASE-12147 Porting Online Config Change from 89-fb].
ifdef::backend-docbook[]
[index]

View File

@ -27,30 +27,32 @@
:icons: font
:experimental:
HBase coprocessors are modeled after the coprocessors which are part of Google's BigTable (link:http://www.scribd.com/doc/21631448/Dean-Keynote-Ladis2009, pages 66-67.). Coprocessors function in a similar way to Linux kernel modules.
HBase coprocessors are modeled after the coprocessors which are part of Google's BigTable (http://www.scribd.com/doc/21631448/Dean-Keynote-Ladis2009, pages 66-67.). Coprocessors function in a similar way to Linux kernel modules.
They provide a way to run server-level code against locally-stored data.
The functionality they provide is very powerful, but also carries great risk and can have adverse effects on the system, at the level of the operating system.
The information in this chapter is primarily sourced and heavily reused from Mingjie Lai's blog post at link:https://blogs.apache.org/hbase/entry/coprocessor_introduction.
The information in this chapter is primarily sourced and heavily reused from Mingjie Lai's blog post at https://blogs.apache.org/hbase/entry/coprocessor_introduction.
Coprocessors are not designed to be used by end users of HBase, but by HBase developers who need to add specialized functionality to HBase.
One example of the use of coprocessors is pluggable compaction and scan policies, which are provided as coprocessors in link:HBASE-6427.
One example of the use of coprocessors is pluggable compaction and scan policies, which are provided as coprocessors in link:https://issues.apache.org/jira/browse/HBASE-6427[HBASE-6427].
== Coprocessor Framework
The implementation of HBase coprocessors diverges from the BigTable implementation.
The HBase framework provides a library and runtime environment for executing user code within the HBase region server and master processes.
The HBase framework provides a library and runtime environment for executing user code within the HBase region server and master processes.
The framework API is provided in the link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/coprocessor/package-summary.html[coprocessor] package.
The framework API is provided in the link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/coprocessor/package-summary.html[coprocessor] package.
Two different types of coprocessors are provided by the framework, based on their scope.
.Types of CoprocessorsSystem Coprocessors::
.Types of Coprocessors
System Coprocessors::
System coprocessors are loaded globally on all tables and regions hosted by a region server.
Table Coprocessors::
You can specify which coprocessors should be loaded on all regions for a table on a per-table basis.
The framework provides two different aspects of extensions as well: [firstterm]_observers_ and [firstterm]_endpoints_.
The framework provides two different aspects of extensions as well: _observers_ and _endpoints_.
Observers::
Observers are analogous to triggers in conventional databases.
@ -80,7 +82,7 @@ You can load the coprocessor from your HBase configuration, so that the coproces
=== Load from Configuration
To configure a coprocessor to be loaded when HBase starts, modify the RegionServer's _hbase-site.xml_ and configure one of the following properties, based on the type of observer you are configuring:
To configure a coprocessor to be loaded when HBase starts, modify the RegionServer's _hbase-site.xml_ and configure one of the following properties, based on the type of observer you are configuring:
* `hbase.coprocessor.region.classes`for RegionObservers and Endpoints
* `hbase.coprocessor.wal.classes`for WALObservers
@ -90,12 +92,12 @@ To configure a coprocessor to be loaded when HBase starts, modify the RegionServ
====
In this example, one RegionObserver is configured for all the HBase tables.
[source,xml]
----
<property>
<name>hbase.coprocessor.region.classes</name>
<value>org.apache.hadoop.hbase.coprocessor.AggregateImplementation</value>
</property>
<name>hbase.coprocessor.region.classes</name>
<value>org.apache.hadoop.hbase.coprocessor.AggregateImplementation</value>
</property>
----
====
@ -106,7 +108,7 @@ Therefore, the jar file must reside on the server-side HBase classpath.
Coprocessors which are loaded in this way will be active on all regions of all tables.
These are the system coprocessor introduced earlier.
The first listed coprocessors will be assigned the priority `Coprocessor.Priority.SYSTEM`.
Each subsequent coprocessor in the list will have its priority value incremented by one (which reduces its priority, because priorities have the natural sort order of Integers).
Each subsequent coprocessor in the list will have its priority value incremented by one (which reduces its priority, because priorities have the natural sort order of Integers).
When calling out to registered observers, the framework executes their callbacks methods in the sorted order of their priority.
Ties are broken arbitrarily.
@ -114,13 +116,12 @@ Ties are broken arbitrarily.
=== Load from the HBase Shell
You can load a coprocessor on a specific table via a table attribute.
The following example will load the [systemitem]+FooRegionObserver+ observer when table [systemitem]+t1+ is read or re-read.
The following example will load the `FooRegionObserver` observer when table `t1` is read or re-read.
.Load a Coprocessor On a Table Using HBase Shell
====
----
hbase(main):005:0> alter 't1', METHOD => 'table_att',
hbase(main):005:0> alter 't1', METHOD => 'table_att',
'coprocessor'=>'hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2'
Updating all regions with the new schema...
1/1 regions updated.
@ -128,18 +129,18 @@ Done.
0 row(s) in 1.0730 seconds
hbase(main):006:0> describe 't1'
DESCRIPTION ENABLED
{NAME => 't1', coprocessor$1 => 'hdfs:///foo.jar|com.foo.FooRegio false
nObserver|1001|arg1=1,arg2=2', FAMILIES => [{NAME => 'c1', DATA_B
LOCK_ENCODING => 'NONE', BLOOMFILTER => 'NONE', REPLICATION_SCOPE
=> '0', VERSIONS => '3', COMPRESSION => 'NONE', MIN_VERSIONS =>
'0', TTL => '2147483647', KEEP_DELETED_CELLS => 'false', BLOCKSIZ
E => '65536', IN_MEMORY => 'false', ENCODE_ON_DISK => 'true', BLO
CKCACHE => 'true'}, {NAME => 'f1', DATA_BLOCK_ENCODING => 'NONE',
BLOOMFILTER => 'NONE', REPLICATION_SCOPE => '0', VERSIONS => '3'
, COMPRESSION => 'NONE', MIN_VERSIONS => '0', TTL => '2147483647'
, KEEP_DELETED_CELLS => 'false', BLOCKSIZE => '65536', IN_MEMORY
=> 'false', ENCODE_ON_DISK => 'true', BLOCKCACHE => 'true'}]}
DESCRIPTION ENABLED
{NAME => 't1', coprocessor$1 => 'hdfs:///foo.jar|com.foo.FooRegio false
nObserver|1001|arg1=1,arg2=2', FAMILIES => [{NAME => 'c1', DATA_B
LOCK_ENCODING => 'NONE', BLOOMFILTER => 'NONE', REPLICATION_SCOPE
=> '0', VERSIONS => '3', COMPRESSION => 'NONE', MIN_VERSIONS =>
'0', TTL => '2147483647', KEEP_DELETED_CELLS => 'false', BLOCKSIZ
E => '65536', IN_MEMORY => 'false', ENCODE_ON_DISK => 'true', BLO
CKCACHE => 'true'}, {NAME => 'f1', DATA_BLOCK_ENCODING => 'NONE',
BLOOMFILTER => 'NONE', REPLICATION_SCOPE => '0', VERSIONS => '3'
, COMPRESSION => 'NONE', MIN_VERSIONS => '0', TTL => '2147483647'
, KEEP_DELETED_CELLS => 'false', BLOCKSIZE => '65536', IN_MEMORY
=> 'false', ENCODE_ON_DISK => 'true', BLOCKCACHE => 'true'}]}
1 row(s) in 0.0190 seconds
----
====
@ -160,7 +161,7 @@ The value contains four pieces of information which are separated by the `|` cha
====
----
hbase(main):007:0> alter 't1', METHOD => 'table_att_unset',
hbase(main):007:0> alter 't1', METHOD => 'table_att_unset',
hbase(main):008:0* NAME => 'coprocessor$1'
Updating all regions with the new schema...
1/1 regions updated.
@ -168,27 +169,27 @@ Done.
0 row(s) in 1.1130 seconds
hbase(main):009:0> describe 't1'
DESCRIPTION ENABLED
{NAME => 't1', FAMILIES => [{NAME => 'c1', DATA_BLOCK_ENCODING => false
'NONE', BLOOMFILTER => 'NONE', REPLICATION_SCOPE => '0', VERSION
S => '3', COMPRESSION => 'NONE', MIN_VERSIONS => '0', TTL => '214
7483647', KEEP_DELETED_CELLS => 'false', BLOCKSIZE => '65536', IN
_MEMORY => 'false', ENCODE_ON_DISK => 'true', BLOCKCACHE => 'true
'}, {NAME => 'f1', DATA_BLOCK_ENCODING => 'NONE', BLOOMFILTER =>
'NONE', REPLICATION_SCOPE => '0', VERSIONS => '3', COMPRESSION =>
'NONE', MIN_VERSIONS => '0', TTL => '2147483647', KEEP_DELETED_C
ELLS => 'false', BLOCKSIZE => '65536', IN_MEMORY => 'false', ENCO
DE_ON_DISK => 'true', BLOCKCACHE => 'true'}]}
DESCRIPTION ENABLED
{NAME => 't1', FAMILIES => [{NAME => 'c1', DATA_BLOCK_ENCODING => false
'NONE', BLOOMFILTER => 'NONE', REPLICATION_SCOPE => '0', VERSION
S => '3', COMPRESSION => 'NONE', MIN_VERSIONS => '0', TTL => '214
7483647', KEEP_DELETED_CELLS => 'false', BLOCKSIZE => '65536', IN
_MEMORY => 'false', ENCODE_ON_DISK => 'true', BLOCKCACHE => 'true
'}, {NAME => 'f1', DATA_BLOCK_ENCODING => 'NONE', BLOOMFILTER =>
'NONE', REPLICATION_SCOPE => '0', VERSIONS => '3', COMPRESSION =>
'NONE', MIN_VERSIONS => '0', TTL => '2147483647', KEEP_DELETED_C
ELLS => 'false', BLOCKSIZE => '65536', IN_MEMORY => 'false', ENCO
DE_ON_DISK => 'true', BLOCKCACHE => 'true'}]}
1 row(s) in 0.0180 seconds
----
====
WARNING: There is no guarantee that the framework will load a given coprocessor successfully.
For example, the shell command neither guarantees a jar file exists at a particular location nor verifies whether the given class is actually contained in the jar file.
For example, the shell command neither guarantees a jar file exists at a particular location nor verifies whether the given class is actually contained in the jar file.
== Check the Status of a Coprocessor
To check the status of a coprocessor after it has been configured, use the +status+ HBase Shell command.
To check the status of a coprocessor after it has been configured, use the `status` HBase Shell command.
----
@ -200,17 +201,17 @@ master coprocessors: []
localhost:52761 1328082515520
requestsPerSecond=3, numberOfOnlineRegions=3, usedHeapMB=32, maxHeapMB=995
-ROOT-,,0
numberOfStores=1, numberOfStorefiles=1, storefileUncompressedSizeMB=0, storefileSizeMB=0, memstoreSizeMB=0,
storefileIndexSizeMB=0, readRequestsCount=54, writeRequestsCount=1, rootIndexSizeKB=0, totalStaticIndexSizeKB=0,
numberOfStores=1, numberOfStorefiles=1, storefileUncompressedSizeMB=0, storefileSizeMB=0, memstoreSizeMB=0,
storefileIndexSizeMB=0, readRequestsCount=54, writeRequestsCount=1, rootIndexSizeKB=0, totalStaticIndexSizeKB=0,
totalStaticBloomSizeKB=0, totalCompactingKVs=0, currentCompactedKVs=0, compactionProgressPct=NaN, coprocessors=[]
.META.,,1
numberOfStores=1, numberOfStorefiles=0, storefileUncompressedSizeMB=0, storefileSizeMB=0, memstoreSizeMB=0,
storefileIndexSizeMB=0, readRequestsCount=97, writeRequestsCount=4, rootIndexSizeKB=0, totalStaticIndexSizeKB=0,
numberOfStores=1, numberOfStorefiles=0, storefileUncompressedSizeMB=0, storefileSizeMB=0, memstoreSizeMB=0,
storefileIndexSizeMB=0, readRequestsCount=97, writeRequestsCount=4, rootIndexSizeKB=0, totalStaticIndexSizeKB=0,
totalStaticBloomSizeKB=0, totalCompactingKVs=0, currentCompactedKVs=0, compactionProgressPct=NaN, coprocessors=[]
t1,,1328082575190.c0491168a27620ffe653ec6c04c9b4d1.
numberOfStores=2, numberOfStorefiles=1, storefileUncompressedSizeMB=0, storefileSizeMB=0, memstoreSizeMB=0,
storefileIndexSizeMB=0, readRequestsCount=0, writeRequestsCount=0, rootIndexSizeKB=0, totalStaticIndexSizeKB=0,
totalStaticBloomSizeKB=0, totalCompactingKVs=0, currentCompactedKVs=0, compactionProgressPct=NaN,
numberOfStores=2, numberOfStorefiles=1, storefileUncompressedSizeMB=0, storefileSizeMB=0, memstoreSizeMB=0,
storefileIndexSizeMB=0, readRequestsCount=0, writeRequestsCount=0, rootIndexSizeKB=0, totalStaticIndexSizeKB=0,
totalStaticBloomSizeKB=0, totalCompactingKVs=0, currentCompactedKVs=0, compactionProgressPct=NaN,
coprocessors=[AggregateImplementation]
0 dead servers
----
@ -218,16 +219,12 @@ coprocessors=[AggregateImplementation]
== Monitor Time Spent in Coprocessors
HBase 0.98.5 introduced the ability to monitor some statistics relating to the amount of time spent executing a given coprocessor.
You can see these statistics via the HBase Metrics framework (see <<hbase_metrics,hbase metrics>> or the Web UI for a given Region Server, via the [label]#Coprocessor Metrics# tab.
You can see these statistics via the HBase Metrics framework (see <<hbase_metrics>> or the Web UI for a given Region Server, via the _Coprocessor Metrics_ tab.
These statistics are valuable for debugging and benchmarking the performance impact of a given coprocessor on your cluster.
Tracked statistics include min, max, average, and 90th, 95th, and 99th percentile.
All times are shown in milliseconds.
The statistics are calculated over coprocessor execution samples recorded during the reporting interval, which is 10 seconds by default.
The metrics sampling rate as described in <<hbase_metrics,hbase metrics>>.
The metrics sampling rate as described in <<hbase_metrics>>.
.Coprocessor Metrics UI
image::coprocessor_stats.png[]
== Status of Coprocessors in HBase
Coprocessors and the coprocessor framework are evolving rapidly and work is ongoing on several different JIRAs.

View File

@ -32,6 +32,7 @@ This is a terminology overlap with relational databases (RDBMSs), but this is no
Instead, it can be helpful to think of an HBase table as a multi-dimensional map.
.HBase Data Model Terminology
Table::
An HBase table consists of multiple rows.
@ -67,26 +68,24 @@ Timestamp::
== Conceptual View
You can read a very understandable explanation of the HBase data model in the blog post link:http://jimbojw.com/wiki/index.php?title=Understanding_Hbase_and_BigTable[Understanding HBase and BigTable] by Jim R. Wilson.
Another good explanation is available in the PDF link:http://0b4af6cdc2f0c5998459-c0245c5c937c5dedcca3f1764ecc9b2f.r43.cf2.rackcdn.com/9353-login1210_khurana.pdf[Introduction
to Basic Schema Design] by Amandeep Khurana.
Another good explanation is available in the PDF link:http://0b4af6cdc2f0c5998459-c0245c5c937c5dedcca3f1764ecc9b2f.r43.cf2.rackcdn.com/9353-login1210_khurana.pdf[Introduction to Basic Schema Design] by Amandeep Khurana.
It may help to read different perspectives to get a solid understanding of HBase schema design.
The linked articles cover the same ground as the information in this section.
The following example is a slightly modified form of the one on page 2 of the link:http://research.google.com/archive/bigtable.html[BigTable] paper.
There is a table called `webtable` that contains two rows (`com.cnn.www` and `com.example.www`), three column families named `contents`, `anchor`, and `people`.
There is a table called `webtable` that contains two rows (`com.cnn.www` and `com.example.www`) and three column families named `contents`, `anchor`, and `people`.
In this example, for the first row (`com.cnn.www`), `anchor` contains two columns (`anchor:cssnsi.com`, `anchor:my.look.ca`) and `contents` contains one column (`contents:html`). This example contains 5 versions of the row with the row key `com.cnn.www`, and one version of the row with the row key `com.example.www`.
The `contents:html` column qualifier contains the entire HTML of a given website.
Qualifiers of the `anchor` column family each contain the external site which links to the site represented by the row, along with the text it used in the anchor of its link.
The `people` column family represents people associated with the site.
The `people` column family represents people associated with the site.
.Column Names
[NOTE]
====
By convention, a column name is made of its column family prefix and a _qualifier_.
For example, the column _contents:html_ is made up of the column family `contents` and the `html` qualifier.
The colon character (`:`) delimits the column family from the column family _qualifier_.
The colon character (`:`) delimits the column family from the column family _qualifier_.
====
.Table `webtable`
@ -109,27 +108,27 @@ This is only a mock-up for illustrative purposes and may not be strictly accurat
[source,json]
----
{
"com.cnn.www": {
contents: {
t6: contents:html: "<html>..."
t5: contents:html: "<html>..."
t3: contents:html: "<html>..."
}
anchor: {
t9: anchor:cnnsi.com = "CNN"
t8: anchor:my.look.ca = "CNN.com"
}
people: {}
}
"com.example.www": {
contents: {
t5: contents:html: "<html>..."
}
anchor: {}
people: {
t5: people:author: "John Doe"
}
}
"com.cnn.www": {
contents: {
t6: contents:html: "<html>..."
t5: contents:html: "<html>..."
t3: contents:html: "<html>..."
}
anchor: {
t9: anchor:cnnsi.com = "CNN"
t8: anchor:my.look.ca = "CNN.com"
}
people: {}
}
"com.example.www": {
contents: {
t5: contents:html: "<html>..."
}
anchor: {}
people: {
t5: people:author: "John Doe"
}
}
}
----
@ -163,18 +162,18 @@ Thus a request for the value of the `contents:html` column at time stamp `t8` wo
Similarly, a request for an `anchor:my.look.ca` value at time stamp `t9` would return no value.
However, if no timestamp is supplied, the most recent value for a particular column would be returned.
Given multiple versions, the most recent is also the first one found, since timestamps are stored in descending order.
Thus a request for the values of all columns in the row `com.cnn.www` if no timestamp is specified would be: the value of `contents:html` from timestamp `t6`, the value of `anchor:cnnsi.com` from timestamp `t9`, the value of `anchor:my.look.ca` from timestamp `t8`.
Thus a request for the values of all columns in the row `com.cnn.www` if no timestamp is specified would be: the value of `contents:html` from timestamp `t6`, the value of `anchor:cnnsi.com` from timestamp `t9`, the value of `anchor:my.look.ca` from timestamp `t8`.
For more information about the internals of how Apache HBase stores data, see <<regions.arch,regions.arch>>.
For more information about the internals of how Apache HBase stores data, see <<regions.arch,regions.arch>>.
== Namespace
A namespace is a logical grouping of tables analogous to a database in relation database systems.
This abstraction lays the groundwork for upcoming multi-tenancy related features:
This abstraction lays the groundwork for upcoming multi-tenancy related features:
* Quota Management (HBASE-8410) - Restrict the amount of resources (ie regions, tables) a namespace can consume.
* Namespace Security Administration (HBASE-9206) - provide another level of security administration for tenants.
* Region server groups (HBASE-6721) - A namespace/table can be pinned onto a subset of regionservers thus guaranteeing a course level of isolation.
* Quota Management (link:https://issues.apache.org/jira/browse/HBASE-8410[HBASE-8410]) - Restrict the amount of resources (ie regions, tables) a namespace can consume.
* Namespace Security Administration (link:https://issues.apache.org/jira/browse/HBASE-9206[HBASE-9206]) - Provide another level of security administration for tenants.
* Region server groups (link:https://issues.apache.org/jira/browse/HBASE-6721[HBASE-6721]) - A namespace/table can be pinned onto a subset of RegionServers thus guaranteeing a course level of isolation.
[[namespace_creation]]
=== Namespace management
@ -221,10 +220,10 @@ alter_namespace 'my_ns', {METHOD => 'set', 'PROPERTY_NAME' => 'PROPERTY_VALUE'}
[[namespace_special]]
=== Predefined namespaces
There are two predefined special namespaces:
There are two predefined special namespaces:
* hbase - system namespace, used to contain hbase internal tables
* default - tables with no explicit specified namespace will automatically fall into this namespace.
* hbase - system namespace, used to contain HBase internal tables
* default - tables with no explicit specified namespace will automatically fall into this namespace
.Examples
====
@ -241,11 +240,11 @@ create 'bar', 'fam'
== Table
Tables are declared up front at schema definition time.
Tables are declared up front at schema definition time.
== Row
Row keys are uninterrpreted bytes.
Row keys are uninterpreted bytes.
Rows are lexicographically sorted with the lowest order appearing first in a table.
The empty byte array is used to denote both the start and end of a tables' namespace.
@ -255,8 +254,7 @@ The empty byte array is used to denote both the start and end of a tables' names
Columns in Apache HBase are grouped into _column families_.
All column members of a column family have the same prefix.
For example, the columns _courses:history_ and _courses:math_ are both members of the _courses_ column family.
The colon character (`:`) delimits the column family from the
column family qualifier.
The colon character (`:`) delimits the column family from the column family qualifier.
The column family prefix must be composed of _printable_ characters.
The qualifying tail, the column family _qualifier_, can be made of any arbitrary bytes.
Column families must be declared up front at schema definition time whereas columns do not need to be defined at schema time but can be conjured on the fly while the table is up an running.
@ -267,29 +265,26 @@ Because tunings and storage specifications are done at the column family level,
== Cells
A _{row, column, version}_ tuple exactly specifies a `cell` in HBase.
Cell content is uninterrpreted bytes
Cell content is uninterpreted bytes
== Data Model Operations
The four primary data model operations are Get, Put, Scan, and Delete.
Operations are applied via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table] instances.
Operations are applied via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table] instances.
=== Get
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html[Get] returns attributes for a specified row.
Gets are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#get(org.apache.hadoop.hbase.client.Get)[
Table.get].
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html[Get] returns attributes for a specified row.
Gets are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#get(org.apache.hadoop.hbase.client.Get)[Table.get].
=== Put
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html[Put] either adds new rows to a table (if the key is new) or can update existing rows (if the key already exists). Puts are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#put(org.apache.hadoop.hbase.client.Put)[
Table.put] (writeBuffer) or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#batch(java.util.List, java.lang.Object[])[
Table.batch] (non-writeBuffer).
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html[Put] either adds new rows to a table (if the key is new) or can update existing rows (if the key already exists). Puts are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#put(org.apache.hadoop.hbase.client.Put)[Table.put] (writeBuffer) or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#batch(java.util.List, java.lang.Object[])[Table.batch] (non-writeBuffer).
[[scan]]
=== Scans
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scan] allow iteration over multiple rows for specified attributes.
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scan] allow iteration over multiple rows for specified attributes.
The following is an example of a Scan on a Table instance.
Assume that a table is populated with rows with keys "row1", "row2", "row3", and then another set of rows with the keys "abc1", "abc2", and "abc3". The following example shows how to set a Scan instance to return the rows beginning with "row".
@ -309,23 +304,24 @@ scan.setRowPrefixFilter(Bytes.toBytes("row"));
ResultScanner rs = table.getScanner(scan);
try {
for (Result r = rs.next(); r != null; r = rs.next()) {
// process result...
// process result...
}
} finally {
rs.close(); // always close the ResultScanner!
}
----
Note that generally the easiest way to specify a specific stop point for a scan is by using the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/InclusiveStopFilter.html[InclusiveStopFilter] class.
Note that generally the easiest way to specify a specific stop point for a scan is by using the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/InclusiveStopFilter.html[InclusiveStopFilter] class.
=== Delete
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Delete.html[Delete] removes a row from a table.
Deletes are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#delete(org.apache.hadoop.hbase.client.Delete)[
HTable.delete].
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Delete.html[Delete] removes a row from a table.
Deletes are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#delete(org.apache.hadoop.hbase.client.Delete)[HTable.delete].
HBase does not modify data in place, and so deletes are handled by creating new markers called _tombstones_.
These tombstones, along with the dead values, are cleaned up on major compactions.
These tombstones, along with the dead values, are cleaned up on major compactions.
See <<version.delete,version.delete>> for more information on deleting versions of columns, and see <<compaction,compaction>> for more information on compactions.
See <<version.delete,version.delete>> for more information on deleting versions of columns, and see <<compaction,compaction>> for more information on compactions.
[[versions]]
== Versions
@ -345,20 +341,20 @@ In particular:
* It is OK to write cells in a non-increasing version order.
Below we describe how the version dimension in HBase currently works.
See link:https://issues.apache.org/jira/browse/HBASE-2406[HBASE-2406] for discussion of HBase versions. link:http://outerthought.org/blog/417-ot.html[Bending time in HBase] makes for a good read on the version, or time, dimension in HBase.
See link:https://issues.apache.org/jira/browse/HBASE-2406[HBASE-2406] for discussion of HBase versions. link:http://outerthought.org/blog/417-ot.html[Bending time in HBase] makes for a good read on the version, or time, dimension in HBase.
It has more detail on versioning than is provided here.
As of this writing, the limiitation _Overwriting values at existing timestamps_ mentioned in the article no longer holds in HBase.
As of this writing, the limitation _Overwriting values at existing timestamps_ mentioned in the article no longer holds in HBase.
This section is basically a synopsis of this article by Bruno Dumon.
[[specify.number.of.versions]]
=== Specifying the Number of Versions to Store
The maximum number of versions to store for a given column is part of the column schema and is specified at table creation, or via an +alter+ command, via `HColumnDescriptor.DEFAULT_VERSIONS`.
The maximum number of versions to store for a given column is part of the column schema and is specified at table creation, or via an `alter` command, via `HColumnDescriptor.DEFAULT_VERSIONS`.
Prior to HBase 0.96, the default number of versions kept was `3`, but in 0.96 and newer has been changed to `1`.
.Modify the Maximum Number of Versions for a Column
.Modify the Maximum Number of Versions for a Column Family
====
This example uses HBase Shell to keep a maximum of 5 versions of column `f1`.
This example uses HBase Shell to keep a maximum of 5 versions of all columns in column family `f1`.
You could also use link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor].
----
@ -366,11 +362,11 @@ hbase> alter t1, NAME => f1, VERSIONS => 5
----
====
.Modify the Minimum Number of Versions for a Column
.Modify the Minimum Number of Versions for a Column Family
====
You can also specify the minimum number of versions to store.
You can also specify the minimum number of versions to store per column family.
By default, this is set to 0, which means the feature is disabled.
The following example sets the minimum number of versions on field `f1` to `2`, via HBase Shell.
The following example sets the minimum number of versions on all columns in column family `f1` to `2`, via HBase Shell.
You could also use link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor].
----
@ -378,7 +374,7 @@ hbase> alter t1, NAME => f1, MIN_VERSIONS => 2
----
====
Starting with HBase 0.98.2, you can specify a global default for the maximum number of versions kept for all newly-created columns, by setting +hbase.column.max.version+ in _hbase-site.xml_.
Starting with HBase 0.98.2, you can specify a global default for the maximum number of versions kept for all newly-created columns, by setting `hbase.column.max.version` in _hbase-site.xml_.
See <<hbase.column.max.version,hbase.column.max.version>>.
[[versions.ops]]
@ -389,13 +385,12 @@ In this section we look at the behavior of the version dimension for each of the
==== Get/Scan
Gets are implemented on top of Scans.
The below discussion of link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html[Get] applies equally to link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scans].
The below discussion of link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html[Get] applies equally to link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scans].
By default, i.e.
if you specify no explicit version, when doing a `get`, the cell whose version has the largest value is returned (which may or may not be the latest one written, see later). The default behavior can be modified in the following ways:
By default, i.e. if you specify no explicit version, when doing a `get`, the cell whose version has the largest value is returned (which may or may not be the latest one written, see later). The default behavior can be modified in the following ways:
* to return more than one version, see link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html#setMaxVersions()[Get.setMaxVersions()]
* to return versions other than the latest, see link:???[Get.setTimeRange()]
* to return versions other than the latest, see link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html#setTimeRange(long,%20long)[Get.setTimeRange()]
+
To retrieve the latest version that is less than or equal to a given value, thus giving the 'latest' state of the record at a certain point in time, just use a range from 0 to the desired version and set the max versions to 1.
@ -438,7 +433,7 @@ Doing a put always creates a new version of a `cell`, at a certain timestamp.
By default the system uses the server's `currentTimeMillis`, but you can specify the version (= the long integer) yourself, on a per-column level.
This means you could assign a time in the past or the future, or use the long value for non-time purposes.
To overwrite an existing value, do a put at exactly the same row, column, and version as that of the cell you would overshadow.
To overwrite an existing value, do a put at exactly the same row, column, and version as that of the cell you want to overwrite.
===== Implicit Version Example
@ -471,42 +466,39 @@ put.add(CF, ATTR, explicitTimeInMs, Bytes.toBytes(data));
table.put(put);
----
Caution: the version timestamp is internally by HBase for things like time-to-live calculations.
Caution: the version timestamp is used internally by HBase for things like time-to-live calculations.
It's usually best to avoid setting this timestamp yourself.
Prefer using a separate timestamp attribute of the row, or have the timestamp a part of the rowkey, or both.
Prefer using a separate timestamp attribute of the row, or have the timestamp as a part of the row key, or both.
[[version.delete]]
==== Delete
There are three different types of internal delete markers.
See Lars Hofhansl's blog for discussion of his attempt adding another, link:http://hadoop-hbase.blogspot.com/2012/01/scanning-in-hbase.html[Scanning
in HBase: Prefix Delete Marker].
See Lars Hofhansl's blog for discussion of his attempt adding another, link:http://hadoop-hbase.blogspot.com/2012/01/scanning-in-hbase.html[Scanning in HBase: Prefix Delete Marker].
* Delete: for a specific version of a column.
* Delete column: for all versions of a column.
* Delete family: for all columns of a particular ColumnFamily
When deleting an entire row, HBase will internally create a tombstone for each ColumnFamily (i.e., not each individual column).
When deleting an entire row, HBase will internally create a tombstone for each ColumnFamily (i.e., not each individual column).
Deletes work by creating _tombstone_ markers.
For example, let's suppose we want to delete a row.
For this you can specify a version, or else by default the `currentTimeMillis` is used.
What this means is [quote]_delete all
cells where the version is less than or equal to this version_.
What this means is _delete all cells where the version is less than or equal to this version_.
HBase never modifies data in place, so for example a delete will not immediately delete (or mark as deleted) the entries in the storage file that correspond to the delete condition.
Rather, a so-called _tombstone_ is written, which will mask the deleted values.
When HBase does a major compaction, the tombstones are processed to actually remove the dead values, together with the tombstones themselves.
If the version you specified when deleting a row is larger than the version of any value in the row, then you can consider the complete row to be deleted.
For an informative discussion on how deletes and versioning interact, see the thread link:http://comments.gmane.org/gmane.comp.java.hadoop.hbase.user/28421[Put w/
timestamp -> Deleteall -> Put w/ timestamp fails] up on the user mailing list.
For an informative discussion on how deletes and versioning interact, see the thread link:http://comments.gmane.org/gmane.comp.java.hadoop.hbase.user/28421[Put w/timestamp -> Deleteall -> Put w/ timestamp fails] up on the user mailing list.
Also see <<keyvalue,keyvalue>> for more information on the internal KeyValue format.
Also see <<keyvalue,keyvalue>> for more information on the internal KeyValue format.
Delete markers are purged during the next major compaction of the store, unless the +KEEP_DELETED_CELLS+ option is set in the column family.
Delete markers are purged during the next major compaction of the store, unless the `KEEP_DELETED_CELLS` option is set in the column family.
To keep the deletes for a configurable amount of time, you can set the delete TTL via the +hbase.hstore.time.to.purge.deletes+ property in _hbase-site.xml_.
If +hbase.hstore.time.to.purge.deletes+ is not set, or set to 0, all delete markers, including those with timestamps in the future, are purged during the next major compaction.
Otherwise, a delete marker with a timestamp in the future is kept until the major compaction which occurs after the time represented by the marker's timestamp plus the value of +hbase.hstore.time.to.purge.deletes+, in milliseconds.
If `hbase.hstore.time.to.purge.deletes` is not set, or set to 0, all delete markers, including those with timestamps in the future, are purged during the next major compaction.
Otherwise, a delete marker with a timestamp in the future is kept until the major compaction which occurs after the time represented by the marker's timestamp plus the value of `hbase.hstore.time.to.purge.deletes`, in milliseconds.
NOTE: This behavior represents a fix for an unexpected change that was introduced in HBase 0.94, and was fixed in link:https://issues.apache.org/jira/browse/HBASE-10118[HBASE-10118].
The change has been backported to HBase 0.94 and newer branches.
@ -529,35 +521,34 @@ But they can occur even if you do not care about time: just do delete and put im
[[major.compactions.change.query.results]]
==== Major compactions change query results
[quote]_...create three cell versions at t1, t2 and t3, with a maximum-versions
setting of 2. So when getting all versions, only the values at t2 and t3 will be
returned. But if you delete the version at t2 or t3, the one at t1 will appear again.
Obviously, once a major compaction has run, such behavior will not be the case
anymore..._ (See _Garbage Collection_ in link:http://outerthought.org/blog/417-ot.html[Bending time in
HBase].)
_...create three cell versions at t1, t2 and t3, with a maximum-versions
setting of 2. So when getting all versions, only the values at t2 and t3 will be
returned. But if you delete the version at t2 or t3, the one at t1 will appear again.
Obviously, once a major compaction has run, such behavior will not be the case
anymore..._ (See _Garbage Collection_ in link:http://outerthought.org/blog/417-ot.html[Bending time in HBase].)
[[dm.sort]]
== Sort Order
All data model operations HBase return data in sorted order.
First by row, then by ColumnFamily, followed by column qualifier, and finally timestamp (sorted in reverse, so newest records are returned first).
First by row, then by ColumnFamily, followed by column qualifier, and finally timestamp (sorted in reverse, so newest records are returned first).
[[dm.column.metadata]]
== Column Metadata
There is no store of column metadata outside of the internal KeyValue instances for a ColumnFamily.
Thus, while HBase can support not only a wide number of columns per row, but a heterogenous set of columns between rows as well, it is your responsibility to keep track of the column names.
Thus, while HBase can support not only a wide number of columns per row, but a heterogeneous set of columns between rows as well, it is your responsibility to keep track of the column names.
The only way to get a complete set of columns that exist for a ColumnFamily is to process all the rows.
For more information about how HBase stores data internally, see <<keyvalue,keyvalue>>.
For more information about how HBase stores data internally, see <<keyvalue,keyvalue>>.
== Joins
Whether HBase supports joins is a common question on the dist-list, and there is a simple answer: it doesn't, at not least in the way that RDBMS' support them (e.g., with equi-joins or outer-joins in SQL). As has been illustrated in this chapter, the read data model operations in HBase are Get and Scan.
Whether HBase supports joins is a common question on the dist-list, and there is a simple answer: it doesn't, at not least in the way that RDBMS' support them (e.g., with equi-joins or outer-joins in SQL). As has been illustrated in this chapter, the read data model operations in HBase are Get and Scan.
However, that doesn't mean that equivalent join functionality can't be supported in your application, but you have to do it yourself.
The two primary strategies are either denormalizing the data upon writing to HBase, or to have lookup tables and do the join between HBase tables in your application or MapReduce code (and as RDBMS' demonstrate, there are several strategies for this depending on the size of the tables, e.g., nested loops vs.
hash-joins). So which is the best approach? It depends on what you are trying to do, and as such there isn't a single answer that works for every use case.
hash-joins). So which is the best approach? It depends on what you are trying to do, and as such there isn't a single answer that works for every use case.
== ACID

View File

@ -28,39 +28,39 @@
:experimental:
This chapter will cover access to Apache HBase either through non-Java languages, or through custom protocols.
For information on using the native HBase APIs, refer to link:http://hbase.apache.org/apidocs/index.html[User API Reference] and the new <<hbase_apis,hbase apis>> chapter.
For information on using the native HBase APIs, refer to link:http://hbase.apache.org/apidocs/index.html[User API Reference] and the new <<hbase_apis,HBase APIs>> chapter.
[[nonjava.jvm]]
== Non-Java Languages Talking to the JVM
Currently the documentation on this topic in the link:http://wiki.apache.org/hadoop/Hbase[Apache HBase Wiki].
See also the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/thrift/package-summary.html#package_description[Thrift API Javadoc].
Currently the documentation on this topic is in the link:http://wiki.apache.org/hadoop/Hbase[Apache HBase Wiki].
See also the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/thrift/package-summary.html#package_description[Thrift API Javadoc].
== REST
Currently most of the documentation on REST exists in the link:http://wiki.apache.org/hadoop/Hbase/Stargate[Apache HBase Wiki on REST] (The REST gateway used to be called 'Stargate'). There are also a nice set of blogs on link:http://blog.cloudera.com/blog/2013/03/how-to-use-the-apache-hbase-rest-interface-part-1/[How-to: Use the Apache HBase REST Interface] by Jesse Anderson.
Currently most of the documentation on REST exists in the link:http://wiki.apache.org/hadoop/Hbase/Stargate[Apache HBase Wiki on REST] (The REST gateway used to be called 'Stargate'). There are also a nice set of blogs on link:http://blog.cloudera.com/blog/2013/03/how-to-use-the-apache-hbase-rest-interface-part-1/[How-to: Use the Apache HBase REST Interface] by Jesse Anderson.
To run your REST server under SSL, set `hbase.rest.ssl.enabled` to `true` and also set the following configs when you launch the REST server: (See example commands in <<jmx_config,JMX config>>)
To run your REST server under SSL, set hbase.rest.ssl.enabled to true and also set the following configs when you launch the REST server:(See example commands in <<jmx_config,JMX config>>)
[source]
----
hbase.rest.ssl.keystore.store
hbase.rest.ssl.keystore.password
hbase.rest.ssl.keystore.keypassword
----
----
HBase ships a simple REST client, see link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/rest/client/package-summary.html[REST client] package for details.
To enable SSL support for it, please also import your certificate into local java cacerts keystore:
To enable SSL support for it, please also import your certificate into local java cacerts keystore:
----
keytool -import -trustcacerts -file /home/user/restserver.cert -keystore $JAVA_HOME/jre/lib/security/cacerts
----
----
== Thrift
Documentation about Thrift has moved to <<thrift,thrift>>.
Documentation about Thrift has moved to <<thrift>>.
[[c]]
== C/C++ Apache HBase Client
FB's Chip Turner wrote a pure C/C++ client.
link:https://github.com/facebook/native-cpp-hbase-client[Check it out].
link:https://github.com/facebook/native-cpp-hbase-client[Check it out].

View File

@ -28,30 +28,31 @@
== Introduction
<<quickstart,quickstart>> will get you up and running on a single-node, standalone instance of HBase, followed by a pseudo-distributed single-machine instance, and finally a fully-distributed cluster.
<<quickstart,Quickstart>> will get you up and running on a single-node, standalone instance of HBase, followed by a pseudo-distributed single-machine instance, and finally a fully-distributed cluster.
[[quickstart]]
== Quick Start
== Quick Start - Standalone HBase
This guide describes setup of a standalone HBase instance running against the local filesystem.
This guide describes the setup of a standalone HBase instance running against the local filesystem.
This is not an appropriate configuration for a production instance of HBase, but will allow you to experiment with HBase.
This section shows you how to create a table in HBase using the +hbase shell+ CLI, insert rows into the table, perform put and scan operations against the table, enable or disable the table, and start and stop HBase.
This section shows you how to create a table in HBase using the `hbase shell` CLI, insert rows into the table, perform put and scan operations against the table, enable or disable the table, and start and stop HBase.
Apart from downloading HBase, this procedure should take less than 10 minutes.
WARNING: Local Filesystem and Durability This is fixed in HBase 0.98.3 and beyond. See link:https://issues.apache.org/jira/browse/HBASE-11272[HBASE-11272] and link:https://issues.apache.org/jira/browse/HBASE-11218[HBASE-11218]._
.Local Filesystem and Durability
WARNING: _The following is fixed in HBase 0.98.3 and beyond. See link:https://issues.apache.org/jira/browse/HBASE-11272[HBASE-11272] and link:https://issues.apache.org/jira/browse/HBASE-11218[HBASE-11218]._
Using HBase with a local filesystem does not guarantee durability.
The HDFS local filesystem implementation will lose edits if files are not properly closed.
This is very likely to happen when you are experimenting with new software, starting and stopping the daemons often and not always cleanly.
You need to run HBase on HDFS to ensure all writes are preserved.
Running against the local filesystem is intended as a shortcut to get you familiar with how the general system works, as the very first phase of evaluation.
See link:https://issues.apache.org/jira/browse/HBASE-3696 and its associated issues for more details about the issues of running on the local filesystem.
See link:https://issues.apache.org/jira/browse/HBASE-3696[HBASE-3696] and its associated issues for more details about the issues of running on the local filesystem.
[[loopback.ip]]
.Loopback IP - HBase 0.94.x and earlier
NOTE: _The below advice is for hbase-0.94.x and older versions only. This is fixed in hbase-0.96.0 and beyond._
Prior to HBase 0.94.x, HBase expected the loopback IP address to be 127.0.0.1. Ubuntu and some other distributions default to 127.0.1.1 and this will cause problems for you . See link:http://blog.devving.com/why-does-hbase-care-about-etchosts/[Why does HBase care about /etc/hosts?] for detail.
Prior to HBase 0.94.x, HBase expected the loopback IP address to be 127.0.0.1. Ubuntu and some other distributions default to 127.0.1.1 and this will cause problems for you.
.Example /etc/hosts File for Ubuntu
@ -69,7 +70,7 @@ The following _/etc/hosts_ file works correctly for HBase 0.94.x and earlier, on
=== JDK Version Requirements
HBase requires that a JDK be installed.
See <<java,java>> for information about supported JDK versions.
See <<java,Java>> for information about supported JDK versions.
=== Get Started with HBase
@ -86,11 +87,11 @@ See <<java,java>> for information about supported JDK versions.
+
----
$ tar xzvf hbase-<?eval ${project.version}?>-hadoop2-bin.tar.gz
$ tar xzvf hbase-<?eval ${project.version}?>-hadoop2-bin.tar.gz
$ cd hbase-<?eval ${project.version}?>-hadoop2/
----
. For HBase 0.98.5 and later, you are required to set the `JAVA_HOME` environment variable before starting HBase.
. For HBase 0.98.5 and later, you are required to set the `JAVA_HOME` environment variable before starting HBase.
Prior to 0.98.5, HBase attempted to detect the location of Java if the variables was not set.
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.
@ -103,14 +104,14 @@ JAVA_HOME=/usr
----
+
NOTE: These instructions assume that each node of your cluster uses the same configuration.
If this is not the case, you may need to set `JAVA_HOME` separately for each node.
If this is not the case, you may need to set `JAVA_HOME` separately for each node.
. Edit _conf/hbase-site.xml_, which is the main HBase configuration file.
At this time, you only need to specify the directory on the local filesystem where HBase and Zookeeper write data.
At this time, you only need to specify the directory on the local filesystem where HBase and ZooKeeper write data.
By default, a new directory is created under /tmp.
Many servers are configured to delete the contents of /tmp upon reboot, so you should store the data elsewhere.
The following configuration will store HBase's data in the _hbase_ directory, in the home directory of the user called [systemitem]+testuser+.
Paste the [markup]+<property>+ tags beneath the [markup]+<configuration>+ tags, which should be empty in a new HBase install.
Many servers are configured to delete the contents of _/tmp_ upon reboot, so you should store the data elsewhere.
The following configuration will store HBase's data in the _hbase_ directory, in the home directory of the user called `testuser`.
Paste the `<property>` tags beneath the `<configuration>` tags, which should be empty in a new HBase install.
+
.Example _hbase-site.xml_ for Standalone HBase
====
@ -136,7 +137,7 @@ If you create the directory, HBase will attempt to do a migration, which is not
. 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.
You can use the +jps+ command to verify that you have one running process called `HMaster`.
You can use the `jps` command to verify that you have one running process called `HMaster`.
In standalone mode HBase runs all daemons within this single JVM, i.e.
the HMaster, a single HRegionServer, and the ZooKeeper daemon.
+
@ -144,10 +145,11 @@ NOTE: Java needs to be installed and available.
If you get an error indicating that Java is not installed, but it is on your system, perhaps in a non-standard location, edit the _conf/hbase-env.sh_ file and modify the `JAVA_HOME` setting to point to the directory that contains _bin/java_ your system.
[[shell_exercises]]
.Procedure: Use HBase For the First Time
. Connect to HBase.
+
Connect to your running instance of HBase using the +hbase shell+ command, located in the _bin/_ directory of your HBase install.
Connect to your running instance of HBase using the `hbase shell` command, located in the [path]_bin/_ directory of your HBase install.
In this example, some usage and version information that is printed when you start HBase Shell has been omitted.
The HBase Shell prompt ends with a `>` character.
+
@ -168,21 +170,21 @@ Use the `create` command to create a new table.
You must specify the table name and the ColumnFamily name.
+
----
hbase(main):001:0> create 'test', 'cf'
0 row(s) in 0.4170 seconds
hbase> create 'test', 'cf'
0 row(s) in 1.2200 seconds
=> Hbase::Table - test
----
. List Information About your Table
+
Use the `list` command to
Use the `list` command to
+
----
hbase> list 'test'
hbase(main):002:0> list 'test'
TABLE
test
1 row(s) in 0.0350 seconds
1 row(s) in 0.0180 seconds
=> ["test"]
----
@ -192,15 +194,14 @@ test
To put data into your table, use the `put` command.
+
----
hbase(main):003:0> put 'test', 'row1', 'cf:a', 'value1'
0 row(s) in 0.0850 seconds
hbase> put 'test', 'row1', 'cf:a', 'value1'
0 row(s) in 0.1770 seconds
hbase(main):004:0> put 'test', 'row2', 'cf:b', 'value2'
0 row(s) in 0.0110 seconds
hbase> put 'test', 'row2', 'cf:b', 'value2'
0 row(s) in 0.0160 seconds
hbase> put 'test', 'row3', 'cf:c', 'value3'
0 row(s) in 0.0260 seconds
hbase(main):005:0> put 'test', 'row3', 'cf:c', 'value3'
0 row(s) in 0.0100 seconds
----
+
Here, we insert three values, one at a time.
@ -210,51 +211,47 @@ Columns in HBase are comprised of a column family prefix, `cf` in this example,
. Scan the table for all data at once.
+
One of the ways to get data from HBase is to scan.
Use the +scan+ command to scan the table for data.
Use the `scan` command to scan the table for data.
You can limit your scan, but for now, all data is fetched.
+
----
hbase> scan 'test'
ROW COLUMN+CELL
row1 column=cf:a, timestamp=1403759475114, value=value1
row2 column=cf:b, timestamp=1403759492807, value=value2
row3 column=cf:c, timestamp=1403759503155, value=value3
3 row(s) in 0.0440 seconds
hbase(main):006:0> scan 'test'
ROW COLUMN+CELL
row1 column=cf:a, timestamp=1421762485768, value=value1
row2 column=cf:b, timestamp=1421762491785, value=value2
row3 column=cf:c, timestamp=1421762496210, value=value3
3 row(s) in 0.0230 seconds
----
. Get a single row of data.
+
To get a single row of data at a time, use the +get+ command.
To get a single row of data at a time, use the `get` command.
+
----
hbase> get 'test', 'row1'
COLUMN CELL
cf:a timestamp=1403759475114, value=value1
1 row(s) in 0.0230 seconds
hbase(main):007:0> get 'test', 'row1'
COLUMN CELL
cf:a timestamp=1421762485768, value=value1
1 row(s) in 0.0350 seconds
----
. Disable a table.
+
If you want to delete a table or change its settings, as well as in some other situations, you need to disable the table first, using the `disable` command.
If you want to delete a table or change its settings, as well as in some other situations, you need to disable the table first, using the `disable` command.
You can re-enable it using the `enable` command.
+
----
hbase(main):008:0> disable 'test'
0 row(s) in 1.1820 seconds
hbase> disable 'test'
0 row(s) in 1.6270 seconds
hbase> enable 'test'
0 row(s) in 0.4500 seconds
hbase(main):009:0> enable 'test'
0 row(s) in 0.1770 seconds
----
+
Disable the table again if you tested the +enable+ command above:
Disable the table again if you tested the `enable` command above:
+
----
hbase> disable 'test'
0 row(s) in 1.6270 seconds
hbase(main):010:0> disable 'test'
0 row(s) in 1.1820 seconds
----
. Drop the table.
@ -262,14 +259,13 @@ hbase> disable 'test'
To drop (delete) a table, use the `drop` command.
+
----
hbase> drop 'test'
0 row(s) in 0.2900 seconds
hbase(main):011:0> drop 'test'
0 row(s) in 0.1370 seconds
----
. Exit the HBase Shell.
+
To exit the HBase Shell and disconnect from your cluster, use the +quit+ command.
To exit the HBase Shell and disconnect from your cluster, use the `quit` command.
HBase is still running in the background.
@ -284,7 +280,7 @@ $
----
. After issuing the command, it can take several minutes for the processes to shut down.
Use the +jps+ to be sure that the HMaster and HRegionServer processes are shut down.
Use the `jps` to be sure that the HMaster and HRegionServer processes are shut down.
[[quickstart_pseudo]]
=== Intermediate - Pseudo-Distributed Local Install
@ -313,7 +309,7 @@ This procedure will create a totally new directory where HBase will store its da
+
Edit the _hbase-site.xml_ configuration.
First, add the following property.
which directs HBase to run in distributed mode, with one JVM instance per daemon.
which directs HBase to run in distributed mode, with one JVM instance per daemon.
+
[source,xml]
----
@ -343,13 +339,13 @@ If you create the directory, HBase will attempt to do a migration, which is not
. Start HBase.
+
Use the _bin/start-hbase.sh_ command to start HBase.
If your system is configured correctly, the +jps+ command should show the HMaster and HRegionServer processes running.
If your system is configured correctly, the `jps` command should show the HMaster and HRegionServer processes running.
. Check the HBase directory in HDFS.
+
If everything worked correctly, HBase created its directory in HDFS.
In the configuration above, it is stored in _/hbase/_ on HDFS.
You can use the +hadoop fs+ command in Hadoop's _bin/_ directory to list this directory.
You can use the `hadoop fs` command in Hadoop's _bin/_ directory to list this directory.
+
----
@ -375,7 +371,7 @@ This step is offered for testing and learning purposes only.
+
The HMaster server controls the HBase cluster.
You can start up to 9 backup HMaster servers, which makes 10 total HMasters, counting the primary.
To start a backup HMaster, use the +local-master-backup.sh+.
To start a backup HMaster, use the `local-master-backup.sh`.
For each backup master you want to start, add a parameter representing the port offset for that master.
Each HMaster uses three ports (16010, 16020, and 16030 by default). The port offset is added to these ports, so using an offset of 2, the backup HMaster would use ports 16012, 16022, and 16032.
The following command starts 3 backup servers using ports 16012/16022/16032, 16013/16023/16033, and 16015/16025/16035.
@ -386,8 +382,8 @@ $ ./bin/local-master-backup.sh 2 3 5
----
+
To kill a backup master without killing the entire cluster, you need to find its process ID (PID). The PID is stored in a file with a name like _/tmp/hbase-USER-X-master.pid_.
The only contents of the file are the PID.
You can use the +kill -9+ command to kill that PID.
The only contents of the file is the PID.
You can use the `kill -9` command to kill that PID.
The following command will kill the master with port offset 1, but leave the cluster running:
+
----
@ -400,8 +396,8 @@ $ cat /tmp/hbase-testuser-1-master.pid |xargs kill -9
The HRegionServer manages the data in its StoreFiles as directed by the HMaster.
Generally, one HRegionServer runs per node in the cluster.
Running multiple HRegionServers on the same system can be useful for testing in pseudo-distributed mode.
The +local-regionservers.sh+ command allows you to run multiple RegionServers.
It works in a similar way to the +local-master-backup.sh+ command, in that each parameter you provide represents the port offset for an instance.
The `local-regionservers.sh` command allows you to run multiple RegionServers.
It works in a similar way to the `local-master-backup.sh` command, in that each parameter you provide represents the port offset for an instance.
Each RegionServer requires two ports, and the default ports are 16020 and 16030.
However, the base ports for additional RegionServers are not the default ports since the default ports are used by the HMaster, which is also a RegionServer since HBase version 1.0.0.
The base ports are 16200 and 16300 instead.
@ -413,7 +409,7 @@ The following command starts four additional RegionServers, running on sequentia
$ .bin/local-regionservers.sh start 2 3 4 5
----
+
To stop a RegionServer manually, use the +local-regionservers.sh+ command with the `stop` parameter and the offset of the server to stop.
To stop a RegionServer manually, use the `local-regionservers.sh` command with the `stop` parameter and the offset of the server to stop.
+
----
$ .bin/local-regionservers.sh stop 3
@ -444,20 +440,21 @@ The architecture will be as follows:
|===
This quickstart assumes that each node is a virtual machine and that they are all on the same network.
It builds upon the previous quickstart, <<quickstart_pseudo,quickstart-pseudo>>, assuming that the system you configured in that procedure is now `node-a`.
Stop HBase on `node-a` before continuing.
It builds upon the previous quickstart, <<quickstart_pseudo>>, assuming that the system you configured in that procedure is now `node-a`.
Stop HBase on `node-a` before continuing.
NOTE: Be sure that all the nodes have full access to communicate, and that no firewall rules are in place which could prevent them from talking to each other.
If you see any errors like `no route to host`, check your firewall.
.Procedure: Configure Password-Less SSH Access
[[passwordless.ssh.quickstart]]
.Procedure: Configure Passwordless SSH Access
`node-a` needs to be able to log into `node-b` and `node-c` (and to itself) in order to start the daemons.
The easiest way to accomplish this is to use the same username on all hosts, and configure password-less SSH login from `node-a` to each of the others.
The easiest way to accomplish this is to use the same username on all hosts, and configure password-less SSH login from `node-a` to each of the others.
. On `node-a`, generate a key pair.
+
While logged in as the user who will run HBase, generate a SSH key pair, using the following command:
While logged in as the user who will run HBase, generate a SSH key pair, using the following command:
+
[source,bash]
----
@ -474,9 +471,9 @@ If it already exists, be aware that it may already contain other keys.
. Copy the public key to the other nodes.
+
Securely copy the public key from `node-a` to each of the nodes, by using the +scp+ or some other secure means.
Securely copy the public key from `node-a` to each of the nodes, by using the `scp` or some other secure means.
On each of the other nodes, create a new file called _.ssh/authorized_keys_ _if it does
not already exist_, and append the contents of the _id_rsa.pub_ file to the end of it.
not already exist_, and append the contents of the _id_rsa.pub_ file to the end of it.
Note that you also need to do this for `node-a` itself.
+
----
@ -485,7 +482,7 @@ $ cat id_rsa.pub >> ~/.ssh/authorized_keys
. Test password-less login.
+
If you performed the procedure correctly, if you SSH from `node-a` to either of the other nodes, using the same username, you should not be prompted for a password.
If you performed the procedure correctly, if you SSH from `node-a` to either of the other nodes, using the same username, you should not be prompted for a password.
. Since `node-b` will run a backup Master, repeat the procedure above, substituting `node-b` everywhere you see `node-a`.
Be sure not to overwrite your existing _.ssh/authorized_keys_ files, but concatenate the new key onto the existing file using the `>>` operator rather than the `>` operator.
@ -515,7 +512,7 @@ This configuration will direct HBase to start and manage a ZooKeeper instance on
+
On `node-a`, edit _conf/hbase-site.xml_ and add the following properties.
+
[source,bourne]
[source,xml]
----
<property>
<name>hbase.zookeeper.quorum</name>
@ -538,24 +535,23 @@ On `node-a`, edit _conf/hbase-site.xml_ and add the following properties.
+
Download and unpack HBase to `node-b`, just as you did for the standalone and pseudo-distributed quickstarts.
. Copy the configuration files from `node-a` to `node-b`.and
`node-c`.
. Copy the configuration files from `node-a` to `node-b`.and `node-c`.
+
Each node of your cluster needs to have the same configuration information.
Copy the contents of the _conf/_ directory to the _conf/_ directory on `node-b` and `node-c`.
Copy the contents of the _conf/_ directory to the _conf/_ directory on `node-b` and `node-c`.
.Procedure: Start and Test Your Cluster
. Be sure HBase is not running on any node.
+
If you forgot to stop HBase from previous testing, you will have errors.
Check to see whether HBase is running on any of your nodes by using the +jps+ command.
Check to see whether HBase is running on any of your nodes by using the `jps` command.
Look for the processes `HMaster`, `HRegionServer`, and `HQuorumPeer`.
If they exist, kill them.
. Start the cluster.
+
On `node-a`, issue the +start-hbase.sh+ command.
On `node-a`, issue the `start-hbase.sh` command.
Your output will be similar to that below.
+
----
@ -566,15 +562,15 @@ node-a.example.com: starting zookeeper, logging to /home/hbuser/hbase-0.98.3-had
node-b.example.com: starting zookeeper, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-zookeeper-node-b.example.com.out
starting master, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-master-node-a.example.com.out
node-c.example.com: starting regionserver, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-regionserver-node-c.example.com.out
node-b.example.com: starting regionserver, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-regionserver-node-b.example.com.out
node-b.example.com: starting regionserver, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-regionserver-node-b.example.com.out
node-b.example.com: starting master, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-master-nodeb.example.com.out
----
+
ZooKeeper starts first, followed by the master, then the RegionServers, and finally the backup masters.
ZooKeeper starts first, followed by the master, then the RegionServers, and finally the backup masters.
. Verify that the processes are running.
+
On each node of the cluster, run the +jps+ command and verify that the correct processes are running on each server.
On each node of the cluster, run the `jps` command and verify that the correct processes are running on each server.
You may see additional Java processes running on your servers as well, if they are used for other purposes.
+
.`node-a` `jps` Output
@ -602,7 +598,7 @@ $ jps
.`node-a` `jps` Output
====
----
$ jps
$ jps
13901 Jps
13639 HQuorumPeer
13737 HRegionServer

View File

@ -28,12 +28,11 @@
:experimental:
This chapter provides information about performing operations using HBase native APIs.
This information is not exhaustive, and provides a quick reference in addition to the link:http://hbase.apache.org/apidocs/index.html[User API
Reference].
This information is not exhaustive, and provides a quick reference in addition to the link:http://hbase.apache.org/apidocs/index.html[User API Reference].
The examples here are not comprehensive or complete, and should be used for purposes of illustration only.
Apache HBase also works with multiple external APIs.
See <<external_apis,external apis>> for more information.
See <<external_apis>> for more information.
== Examples
@ -60,7 +59,7 @@ import static com.example.hbase.Constants.*;
public class CreateSchema {
public static void createOrOverwrite(HBaseAdmin admin, HTableDescriptor table) throws IOException {
public static void createOrOverwrite(HBaseAdmin admin, HTableDescriptor table) throws IOException {
if (admin.tableExists(table.getName())) {
admin.disableTable(table.getName());
admin.deleteTable(table.getName());
@ -85,7 +84,6 @@ public class CreateSchema {
}
}
}
----
====
@ -96,42 +94,41 @@ This example has been tested on HBase 0.96.1.1.
[source,java]
----
public static void upgradeFrom0 (Configuration config) {
try {
final HBaseAdmin admin = new HBaseAdmin(config);
TableName tableName = TableName.valueOf(TABLE_ASSETMETA);
HTableDescriptor table_assetmeta = new HTableDescriptor(tableName);
table_assetmeta.addFamily(new HColumnDescriptor(CF_DEFAULT).setCompressionType(Algorithm.SNAPPY));
try {
final HBaseAdmin admin = new HBaseAdmin(config);
TableName tableName = TableName.valueOf(TABLE_ASSETMETA);
HTableDescriptor table_assetmeta = new HTableDescriptor(tableName);
table_assetmeta.addFamily(new HColumnDescriptor(CF_DEFAULT).setCompressionType(Algorithm.SNAPPY));
// Create a new table.
// Create a new table.
System.out.print("Creating table_assetmeta. ");
admin.createTable(table_assetmeta);
System.out.println(" Done.");
System.out.print("Creating table_assetmeta. ");
admin.createTable(table_assetmeta);
System.out.println(" Done.");
// Update existing table
HColumnDescriptor newColumn = new HColumnDescriptor("NEWCF");
newColumn.setCompactionCompressionType(Algorithm.GZ);
newColumn.setMaxVersions(HConstants.ALL_VERSIONS);
admin.addColumn(tableName, newColumn);
// Update existing table
HColumnDescriptor newColumn = new HColumnDescriptor("NEWCF");
newColumn.setCompactionCompressionType(Algorithm.GZ);
newColumn.setMaxVersions(HConstants.ALL_VERSIONS);
admin.addColumn(tableName, newColumn);
// Disable an existing table
admin.disableTable(tableName);
// Disable an existing table
admin.disableTable(tableName);
// Delete an existing column family
admin.deleteColumn(tableName, CF_DEFAULT);
// Delete an existing column family
admin.deleteColumn(tableName, CF_DEFAULT);
// Delete a table (Need to be disabled first)
admin.deleteTable(tableName);
// Delete a table (Need to be disabled first)
admin.deleteTable(tableName);
admin.close();
} catch (Exception e) {
e.printStackTrace();
System.exit(-1);
}
admin.close();
} catch (Exception e) {
e.printStackTrace();
System.exit(-1);
}
}
----
====

View File

@ -29,48 +29,48 @@
Apache MapReduce is a software framework used to analyze large amounts of data, and is the framework used most often with link:http://hadoop.apache.org/[Apache Hadoop].
MapReduce itself is out of the scope of this document.
A good place to get started with MapReduce is link:http://hadoop.apache.org/docs/r1.2.1/mapred_tutorial.html.
MapReduce version 2 (MR2)is now part of link:http://hadoop.apache.org/docs/r2.3.0/hadoop-yarn/hadoop-yarn-site/[YARN].
A good place to get started with MapReduce is http://hadoop.apache.org/docs/r2.6.0/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html.
MapReduce version 2 (MR2)is now part of link:http://hadoop.apache.org/docs/r2.3.0/hadoop-yarn/hadoop-yarn-site/[YARN].
This chapter discusses specific configuration steps you need to take to use MapReduce on data within HBase.
In addition, it discusses other interactions and issues between HBase and MapReduce jobs.
In addition, it discusses other interactions and issues between HBase and MapReduce jobs.
.mapred and mapreduce
.`mapred` and `mapreduce`
[NOTE]
====
There are two mapreduce packages in HBase as in MapReduce itself: _org.apache.hadoop.hbase.mapred_ and _org.apache.hadoop.hbase.mapreduce_.
The former does old-style API and the latter the new style.
The latter has more facility though you can usually find an equivalent in the older package.
Pick the package that goes with your mapreduce deploy.
Pick the package that goes with your MapReduce deploy.
When in doubt or starting over, pick the _org.apache.hadoop.hbase.mapreduce_.
In the notes below, we refer to o.a.h.h.mapreduce but replace with the o.a.h.h.mapred if that is what you are using.
====
In the notes below, we refer to o.a.h.h.mapreduce but replace with the o.a.h.h.mapred if that is what you are using.
====
[[hbase.mapreduce.classpath]]
== HBase, MapReduce, and the CLASSPATH
By default, MapReduce jobs deployed to a MapReduce cluster do not have access to either the HBase configuration under `$HBASE_CONF_DIR` or the HBase classes.
To give the MapReduce jobs the access they need, you could add _hbase-site.xml_ to the _$HADOOP_HOME/conf/_ directory and add the HBase JARs to the _`$HADOOP_HOME`/conf/_ directory, then copy these changes across your cluster.
You could add hbase-site.xml to `$HADOOP_HOME`/conf and add HBase jars to the $HADOOP_HOME/lib.
You would then need to copy these changes across your cluster or edit _`$HADOOP_HOME`/conf/hadoop-env.sh_ and add them to the `HADOOP_CLASSPATH` variable.
To give the MapReduce jobs the access they need, you could add _hbase-site.xml_ to the _$HADOOP_HOME/conf/_ directory and add the HBase JARs to the _HADOOP_HOME/conf/_ directory, then copy these changes across your cluster.
You could add _hbase-site.xml_ to _$HADOOP_HOME/conf_ and add HBase jars to the _$HADOOP_HOME/lib_ directory.
You would then need to copy these changes across your cluster or edit _$HADOOP_HOMEconf/hadoop-env.sh_ and add them to the `HADOOP_CLASSPATH` variable.
However, this approach is not recommended because it will pollute your Hadoop install with HBase references.
It also requires you to restart the Hadoop cluster before Hadoop can use the HBase data.
Since HBase 0.90.x, HBase adds its dependency JARs to the job configuration itself.
The dependencies only need to be available on the local `CLASSPATH`.
The following example runs the bundled HBase link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html[RowCounter] MapReduce job against a table named [systemitem]+usertable+ If you have not set the environment variables expected in the command (the parts prefixed by a `$` sign and curly braces), you can use the actual system paths instead.
The following example runs the bundled HBase link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html[RowCounter] MapReduce job against a table named `usertable` If you have not set the environment variables expected in the command (the parts prefixed by a `$` sign and curly braces), you can use the actual system paths instead.
Be sure to use the correct version of the HBase JAR for your system.
The backticks (``` symbols) cause ths shell to execute the sub-commands, setting the CLASSPATH as part of the command.
This example assumes you use a BASH-compatible shell.
The backticks (``` symbols) cause ths shell to execute the sub-commands, setting the `CLASSPATH` as part of the command.
This example assumes you use a BASH-compatible shell.
[source,bash]
----
$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` ${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/hbase-server-VERSION.jar rowcounter usertable
----
When the command runs, internally, the HBase JAR finds the dependencies it needs for zookeeper, guava, and its other dependencies on the passed `HADOOP_CLASSPATH` and adds the JARs to the MapReduce job configuration.
See the source at TableMapReduceUtil#addDependencyJars(org.apache.hadoop.mapreduce.Job) for how this is done.
When the command runs, internally, the HBase JAR finds the dependencies it needs for ZooKeeper, Guava, and its other dependencies on the passed `HADOOP_CLASSPATH` and adds the JARs to the MapReduce job configuration.
See the source at `TableMapReduceUtil#addDependencyJars(org.apache.hadoop.mapreduce.Job)` for how this is done.
[NOTE]
====
@ -89,10 +89,10 @@ $ HADOOP_CLASSPATH=${HBASE_HOME}/hbase-server/target/hbase-server-VERSION-SNAPSH
----
====
.Notice to Mapreduce users of HBase 0.96.1 and above
.Notice to MapReduce users of HBase 0.96.1 and above
[CAUTION]
====
Some mapreduce jobs that use HBase fail to launch.
Some MapReduce jobs that use HBase fail to launch.
The symptom is an exception similar to the following:
----
@ -125,15 +125,15 @@ Exception in thread "main" java.lang.IllegalAccessError: class
...
----
This is caused by an optimization introduced in link:https://issues.apache.org/jira/browse/HBASE-9867[HBASE-9867] that inadvertently introduced a classloader dependency.
This is caused by an optimization introduced in link:https://issues.apache.org/jira/browse/HBASE-9867[HBASE-9867] that inadvertently introduced a classloader dependency.
This affects both jobs using the `-libjars` option and "fat jar," those which package their runtime dependencies in a nested `lib` folder.
In order to satisfy the new classloader requirements, hbase-protocol.jar must be included in Hadoop's classpath.
See <<hbase.mapreduce.classpath,hbase.mapreduce.classpath>> for current recommendations for resolving classpath errors.
In order to satisfy the new classloader requirements, `hbase-protocol.jar` must be included in Hadoop's classpath.
See <<hbase.mapreduce.classpath>> for current recommendations for resolving classpath errors.
The following is included for historical purposes.
This can be resolved system-wide by including a reference to the hbase-protocol.jar in hadoop's lib directory, via a symlink or by copying the jar into the new location.
This can be resolved system-wide by including a reference to the `hbase-protocol.jar` in Hadoop's lib directory, via a symlink or by copying the jar into the new location.
This can also be achieved on a per-job launch basis by including it in the `HADOOP_CLASSPATH` environment variable at job submission time.
When launching jobs that package their dependencies, all three of the following job launching commands satisfy this requirement:
@ -162,7 +162,7 @@ This functionality was lost due to a bug in HBase 0.95 (link:https://issues.apac
The priority order for choosing the scanner caching is as follows:
. Caching settings which are set on the scan object.
. Caching settings which are specified via the configuration option +hbase.client.scanner.caching+, which can either be set manually in _hbase-site.xml_ or via the helper method `TableMapReduceUtil.setScannerCaching()`.
. Caching settings which are specified via the configuration option `hbase.client.scanner.caching`, which can either be set manually in _hbase-site.xml_ or via the helper method `TableMapReduceUtil.setScannerCaching()`.
. The default value `HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING`, which is set to `100`.
Optimizing the caching settings is a balance between the time the client waits for a result and the number of sets of results the client needs to receive.
@ -176,7 +176,7 @@ See the API documentation for link:https://hbase.apache.org/apidocs/org/apache/h
== Bundled HBase MapReduce Jobs
The HBase JAR also serves as a Driver for some bundled mapreduce jobs.
The HBase JAR also serves as a Driver for some bundled MapReduce jobs.
To learn about the bundled MapReduce jobs, run the following command.
[source,bash]
@ -202,35 +202,35 @@ $ ${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/hbase-server-VERSION.jar rowcounte
== HBase as a MapReduce Job Data Source and Data Sink
HBase can be used as a data source, link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html[TableInputFormat], and data sink, link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.html[TableOutputFormat] or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.html[MultiTableOutputFormat], for MapReduce jobs.
HBase can be used as a data source, link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html[TableInputFormat], and data sink, link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.html[TableOutputFormat] or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.html[MultiTableOutputFormat], for MapReduce jobs.
Writing MapReduce jobs that read or write HBase, it is advisable to subclass link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableMapper.html[TableMapper] and/or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableReducer.html[TableReducer].
See the do-nothing pass-through classes link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/IdentityTableMapper.html[IdentityTableMapper] and link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/IdentityTableReducer.html[IdentityTableReducer] for basic usage.
For a more involved example, see link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html[RowCounter] or review the `org.apache.hadoop.hbase.mapreduce.TestTableMapReduce` unit test.
See the do-nothing pass-through classes link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/IdentityTableMapper.html[IdentityTableMapper] and link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/IdentityTableReducer.html[IdentityTableReducer] for basic usage.
For a more involved example, see link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html[RowCounter] or review the `org.apache.hadoop.hbase.mapreduce.TestTableMapReduce` unit test.
If you run MapReduce jobs that use HBase as source or sink, need to specify source and sink table and column names in your configuration.
When you read from HBase, the `TableInputFormat` requests the list of regions from HBase and makes a map, which is either a `map-per-region` or `mapreduce.job.maps` map, whichever is smaller.
If your job only has two maps, raise `mapreduce.job.maps` to a number greater than the number of regions.
Maps will run on the adjacent TaskTracker if you are running a TaskTracer and RegionServer per node.
Maps will run on the adjacent TaskTracker/NodeManager if you are running a TaskTracer/NodeManager and RegionServer per node.
When writing to HBase, it may make sense to avoid the Reduce step and write back into HBase from within your map.
This approach works when your job does not need the sort and collation that MapReduce does on the map-emitted data.
On insert, HBase 'sorts' so there is no point double-sorting (and shuffling data around your MapReduce cluster) unless you need to.
If you do not need the Reduce, you myour map might emit counts of records processed for reporting at the end of the jobj, or set the number of Reduces to zero and use TableOutputFormat.
If you do not need the Reduce, your map might emit counts of records processed for reporting at the end of the job, or set the number of Reduces to zero and use TableOutputFormat.
If running the Reduce step makes sense in your case, you should typically use multiple reducers so that load is spread across the HBase cluster.
A new HBase partitioner, the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.html[HRegionPartitioner], can run as many reducers the number of existing regions.
The HRegionPartitioner is suitable when your table is large and your upload will not greatly alter the number of existing regions upon completion.
Otherwise use the default partitioner.
Otherwise use the default partitioner.
== Writing HFiles Directly During Bulk Import
If you are importing into a new table, you can bypass the HBase API and write your content directly to the filesystem, formatted into HBase data files (HFiles). Your import will run faster, perhaps an order of magnitude faster.
For more on how this mechanism works, see <<arch.bulk.load,arch.bulk.load>>.
For more on how this mechanism works, see <<arch.bulk.load>>.
== RowCounter Example
The included link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html[RowCounter] MapReduce job uses `TableInputFormat` and does a count of all rows in the specified table.
To run it, use the following command:
The included link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html[RowCounter] MapReduce job uses `TableInputFormat` and does a count of all rows in the specified table.
To run it, use the following command:
[source,bash]
----
@ -239,9 +239,9 @@ $ ./bin/hadoop jar hbase-X.X.X.jar
This will invoke the HBase MapReduce Driver class.
Select `rowcounter` from the choice of jobs offered.
This will print rowcouner usage advice to standard output.
This will print rowcounter usage advice to standard output.
Specify the tablename, column to count, and output directory.
If you have classpath errors, see <<hbase.mapreduce.classpath,hbase.mapreduce.classpath>>.
If you have classpath errors, see <<hbase.mapreduce.classpath>>.
[[splitter]]
== Map-Task Splitting
@ -249,14 +249,14 @@ If you have classpath errors, see <<hbase.mapreduce.classpath,hbase.mapreduce.cl
[[splitter.default]]
=== The Default HBase MapReduce Splitter
When link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html[TableInputFormat] is used to source an HBase table in a MapReduce job, its splitter will make a map task for each region of the table.
When link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html[TableInputFormat] is used to source an HBase table in a MapReduce job, its splitter will make a map task for each region of the table.
Thus, if there are 100 regions in the table, there will be 100 map-tasks for the job - regardless of how many column families are selected in the Scan.
[[splitter.custom]]
=== Custom Splitters
For those interested in implementing custom splitters, see the method `getSplits` in link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.html[TableInputFormatBase].
That is where the logic for map-task assignment resides.
That is where the logic for map-task assignment resides.
[[mapreduce.example]]
== HBase MapReduce Examples
@ -325,16 +325,16 @@ scan.setCacheBlocks(false); // don't set to true for MR jobs
// set other scan attrs
TableMapReduceUtil.initTableMapperJob(
sourceTable, // input table
scan, // Scan instance to control CF and attribute selection
MyMapper.class, // mapper class
null, // mapper output key
null, // mapper output value
job);
sourceTable, // input table
scan, // Scan instance to control CF and attribute selection
MyMapper.class, // mapper class
null, // mapper output key
null, // mapper output value
job);
TableMapReduceUtil.initTableReducerJob(
targetTable, // output table
null, // reducer class
job);
targetTable, // output table
null, // reducer class
job);
job.setNumReduceTasks(0);
boolean b = job.waitForCompletion(true);
@ -343,45 +343,45 @@ if (!b) {
}
----
An explanation is required of what `TableMapReduceUtil` is doing, especially with the reducer. link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.html[TableOutputFormat] is being used as the outputFormat class, and several parameters are being set on the config (e.g., TableOutputFormat.OUTPUT_TABLE), as well as setting the reducer output key to `ImmutableBytesWritable` and reducer value to `Writable`.
An explanation is required of what `TableMapReduceUtil` is doing, especially with the reducer. link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.html[TableOutputFormat] is being used as the outputFormat class, and several parameters are being set on the config (e.g., `TableOutputFormat.OUTPUT_TABLE`), as well as setting the reducer output key to `ImmutableBytesWritable` and reducer value to `Writable`.
These could be set by the programmer on the job and conf, but `TableMapReduceUtil` tries to make things easier.
The following is the example mapper, which will create a `Put` and matching the input `Result` and emit it.
Note: this is what the CopyTable utility does.
The following is the example mapper, which will create a `Put` and matching the input `Result` and emit it.
Note: this is what the CopyTable utility does.
[source,java]
----
public static class MyMapper extends TableMapper<ImmutableBytesWritable, Put> {
public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException, InterruptedException {
// this example is just copying the data from the source table...
context.write(row, resultToPut(row,value));
}
public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException, InterruptedException {
// this example is just copying the data from the source table...
context.write(row, resultToPut(row,value));
}
private static Put resultToPut(ImmutableBytesWritable key, Result result) throws IOException {
Put put = new Put(key.get());
for (KeyValue kv : result.raw()) {
put.add(kv);
}
return 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);
}
return put;
}
}
----
There isn't actually a reducer step, so `TableOutputFormat` takes care of sending the `Put` to the target table.
There isn't actually a reducer step, so `TableOutputFormat` takes care of sending the `Put` to the target table.
This is just an example, developers could choose not to use `TableOutputFormat` and connect to the target table themselves.
This is just an example, developers could choose not to use `TableOutputFormat` and connect to the target table themselves.
[[mapreduce.example.readwrite.multi]]
=== HBase MapReduce Read/Write Example With Multi-Table Output
TODO: example for `MultiTableOutputFormat`.
TODO: example for `MultiTableOutputFormat`.
[[mapreduce.example.summary]]
=== HBase MapReduce Summary to HBase Example
The following example uses HBase as a MapReduce source and sink with a summarization step.
This example will count the number of distinct instances of a value in a table and write those summarized counts in another table.
This example will count the number of distinct instances of a value in a table and write those summarized counts in another table.
[source,java]
----
@ -395,72 +395,71 @@ scan.setCacheBlocks(false); // don't set to true for MR jobs
// set other scan attrs
TableMapReduceUtil.initTableMapperJob(
sourceTable, // input table
scan, // Scan instance to control CF and attribute selection
MyMapper.class, // mapper class
Text.class, // mapper output key
IntWritable.class, // mapper output value
job);
sourceTable, // input table
scan, // Scan instance to control CF and attribute selection
MyMapper.class, // mapper class
Text.class, // mapper output key
IntWritable.class, // mapper output value
job);
TableMapReduceUtil.initTableReducerJob(
targetTable, // output table
MyTableReducer.class, // reducer class
job);
targetTable, // output table
MyTableReducer.class, // reducer class
job);
job.setNumReduceTasks(1); // at least one, adjust as required
boolean b = job.waitForCompletion(true);
if (!b) {
throw new IOException("error with job!");
throw new IOException("error with job!");
}
----
----
In this example mapper a column with a String-value is chosen as the value to summarize upon.
This value is used as the key to emit from the mapper, and an `IntWritable` represents an instance counter.
This value is used as the key to emit from the mapper, and an `IntWritable` represents an instance counter.
[source,java]
----
public static class MyMapper extends TableMapper<Text, IntWritable> {
public static final byte[] CF = "cf".getBytes();
public static final byte[] ATTR1 = "attr1".getBytes();
public static final byte[] CF = "cf".getBytes();
public static final byte[] ATTR1 = "attr1".getBytes();
private final IntWritable ONE = new IntWritable(1);
private Text text = new Text();
private final IntWritable ONE = new IntWritable(1);
private Text text = new Text();
public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException, InterruptedException {
String val = new String(value.getValue(CF, ATTR1));
text.set(val); // we can only emit Writables...
context.write(text, ONE);
}
public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException, InterruptedException {
String val = new String(value.getValue(CF, ATTR1));
text.set(val); // we can only emit Writables...
context.write(text, ONE);
}
}
----
----
In the reducer, the "ones" are counted (just like any other MR example that does this), and then emits a `Put`.
In the reducer, the "ones" are counted (just like any other MR example that does this), and then emits a `Put`.
[source,java]
----
public static class MyTableReducer extends TableReducer<Text, IntWritable, ImmutableBytesWritable> {
public static final byte[] CF = "cf".getBytes();
public static final byte[] COUNT = "count".getBytes();
public static final byte[] CF = "cf".getBytes();
public static final byte[] COUNT = "count".getBytes();
public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException {
int i = 0;
for (IntWritable val : values) {
i += val.get();
}
Put put = new Put(Bytes.toBytes(key.toString()));
put.add(CF, COUNT, Bytes.toBytes(i));
public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException {
int i = 0;
for (IntWritable val : values) {
i += val.get();
}
Put put = new Put(Bytes.toBytes(key.toString()));
put.add(CF, COUNT, Bytes.toBytes(i));
context.write(null, put);
}
context.write(null, put);
}
}
----
----
[[mapreduce.example.summary.file]]
=== HBase MapReduce Summary to File Example
This very similar to the summary example above, with exception that this is using HBase as a MapReduce source but HDFS as the sink.
The differences are in the job setup and in the reducer.
The mapper remains the same.
The mapper remains the same.
[source,java]
----
@ -474,19 +473,19 @@ scan.setCacheBlocks(false); // don't set to true for MR jobs
// set other scan attrs
TableMapReduceUtil.initTableMapperJob(
sourceTable, // input table
scan, // Scan instance to control CF and attribute selection
MyMapper.class, // mapper class
Text.class, // mapper output key
IntWritable.class, // mapper output value
job);
sourceTable, // input table
scan, // Scan instance to control CF and attribute selection
MyMapper.class, // mapper class
Text.class, // mapper output key
IntWritable.class, // mapper output value
job);
job.setReducerClass(MyReducer.class); // reducer class
job.setNumReduceTasks(1); // at least one, adjust as required
FileOutputFormat.setOutputPath(job, new Path("/tmp/mr/mySummaryFile")); // adjust directories as required
boolean b = job.waitForCompletion(true);
if (!b) {
throw new IOException("error with job!");
throw new IOException("error with job!");
}
----
@ -497,68 +496,68 @@ As for the Reducer, it is a "generic" Reducer instead of extending TableMapper a
----
public static class MyReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException {
int i = 0;
for (IntWritable val : values) {
i += val.get();
}
context.write(key, new IntWritable(i));
}
public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException {
int i = 0;
for (IntWritable val : values) {
i += val.get();
}
context.write(key, new IntWritable(i));
}
}
----
[[mapreduce.example.summary.noreducer]]
=== HBase MapReduce Summary to HBase Without Reducer
It is also possible to perform summaries without a reducer - if you use HBase as the reducer.
It is also possible to perform summaries without a reducer - if you use HBase as the reducer.
An HBase target table would need to exist for the job summary.
The Table method `incrementColumnValue` would be used to atomically increment values.
From a performance perspective, it might make sense to keep a Map of values with their values to be incremeneted for each map-task, and make one update per key at during the `cleanup` method of the mapper.
However, your milage may vary depending on the number of rows to be processed and unique keys.
From a performance perspective, it might make sense to keep a Map of values with their values to be incremented for each map-task, and make one update per key at during the `cleanup` method of the mapper.
However, your mileage may vary depending on the number of rows to be processed and unique keys.
In the end, the summary results are in HBase.
In the end, the summary results are in HBase.
[[mapreduce.example.summary.rdbms]]
=== HBase MapReduce Summary to RDBMS
Sometimes it is more appropriate to generate summaries to an RDBMS.
For these cases, it is possible to generate summaries directly to an RDBMS via a custom reducer.
The `setup` method can connect to an RDBMS (the connection information can be passed via custom parameters in the context) and the cleanup method can close the connection.
The `setup` method can connect to an RDBMS (the connection information can be passed via custom parameters in the context) and the cleanup method can close the connection.
It is critical to understand that number of reducers for the job affects the summarization implementation, and you'll have to design this into your reducer.
Specifically, whether it is designed to run as a singleton (one reducer) or multiple reducers.
Neither is right or wrong, it depends on your use-case.
Recognize that the more reducers that are assigned to the job, the more simultaneous connections to the RDBMS will be created - this will scale, but only to a point.
Recognize that the more reducers that are assigned to the job, the more simultaneous connections to the RDBMS will be created - this will scale, but only to a point.
[source,java]
----
public static class MyRdbmsReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
public static class MyRdbmsReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
private Connection c = null;
private Connection c = null;
public void setup(Context context) {
// create DB connection...
}
public void setup(Context context) {
// create DB connection...
}
public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException {
// do summarization
// in this example the keys are Text, but this is just an example
}
public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException {
// do summarization
// in this example the keys are Text, but this is just an example
}
public void cleanup(Context context) {
// close db connection
}
public void cleanup(Context context) {
// close db connection
}
}
----
In the end, the summary results are written to your RDBMS table/s.
In the end, the summary results are written to your RDBMS table/s.
[[mapreduce.htable.access]]
== Accessing Other HBase Tables in a MapReduce Job
Although the framework currently allows one HBase table as input to a MapReduce job, other HBase tables can be accessed as lookup tables, etc., in a MapReduce job via creating an Table instance in the setup method of the Mapper.
Although the framework currently allows one HBase table as input to a MapReduce job, other HBase tables can be accessed as lookup tables, etc., in a MapReduce job via creating an Table instance in the setup method of the Mapper.
[source,java]
----
public class MyMapper extends TableMapper<Text, LongWritable> {
@ -571,16 +570,16 @@ public class MyMapper extends TableMapper<Text, LongWritable> {
}
public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException, InterruptedException {
// process Result...
// use 'myOtherTable' for lookups
// process Result...
// use 'myOtherTable' for lookups
}
----
----
[[mapreduce.specex]]
== Speculative Execution
It is generally advisable to turn off speculative execution for MapReduce jobs that use HBase as a source.
This can either be done on a per-Job basis through properties, on on the entire cluster.
Especially for longer running jobs, speculative execution will create duplicate map-tasks which will double-write your data to HBase; this is probably not what you want.
Especially for longer running jobs, speculative execution will create duplicate map-tasks which will double-write your data to HBase; this is probably not what you want.
See <<spec.ex,spec.ex>> for more information.
See <<spec.ex,spec.ex>> for more information.

View File

@ -28,7 +28,7 @@
:experimental:
This chapter will cover operational tools and practices required of a running Apache HBase cluster.
The subject of operations is related to the topics of <<trouble,trouble>>, <<performance,performance>>, and <<configuration,configuration>> but is a distinct topic in itself.
The subject of operations is related to the topics of <<trouble>>, <<performance>>, and <<configuration>> but is a distinct topic in itself.
[[tools]]
== HBase Tools and Utilities
@ -36,9 +36,9 @@ The subject of operations is related to the topics of <<trouble,trouble>>, <<per
HBase provides several tools for administration, analysis, and debugging of your cluster.
The entry-point to most of these tools is the _bin/hbase_ command, though some tools are available in the _dev-support/_ directory.
To see usage instructions for _bin/hbase_ command, run it with no arguments, or with the +-h+ argument.
To see usage instructions for _bin/hbase_ command, run it with no arguments, or with the `-h` argument.
These are the usage instructions for HBase 0.98.x.
Some commands, such as +version+, +pe+, +ltt+, +clean+, are not available in previous versions.
Some commands, such as `version`, `pe`, `ltt`, `clean`, are not available in previous versions.
----
$ bin/hbase
@ -51,7 +51,7 @@ Commands:
Some commands take arguments. Pass no args or -h for usage.
shell Run the HBase shell
hbck Run the hbase 'fsck' tool
hlog Write-ahead-log analyzer
wal Write-ahead-log analyzer
hfile Store file analyzer
zkcli Run the ZooKeeper shell
upgrade Upgrade hbase
@ -71,13 +71,12 @@ Some commands take arguments. Pass no args or -h for usage.
----
Some of the tools and utilities below are Java classes which are passed directly to the _bin/hbase_ command, as referred to in the last line of the usage instructions.
Others, such as +hbase shell+ (<<shell,shell>>), +hbase upgrade+ (<<upgrading,upgrading>>), and +hbase
thrift+ (<<thrift,thrift>>), are documented elsewhere in this guide.
Others, such as `hbase shell` (<<shell>>), `hbase upgrade` (<<upgrading>>), and `hbase thrift` (<<thrift>>), are documented elsewhere in this guide.
=== Canary
There is a Canary class can help users to canary-test the HBase cluster status, with every column-family for every regions or regionservers granularity.
To see the usage, use the `--help` parameter.
There is a Canary class can help users to canary-test the HBase cluster status, with every column-family for every regions or RegionServer's granularity.
To see the usage, use the `--help` parameter.
----
$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -help
@ -96,7 +95,7 @@ Usage: bin/hbase org.apache.hadoop.hbase.tool.Canary [opts] [table1 [table2]...]
----
This tool will return non zero error codes to user for collaborating with other monitoring tools, such as Nagios.
The error code definitions are:
The error code definitions are:
[source,java]
----
@ -107,26 +106,26 @@ private static final int ERROR_EXIT_CODE = 4;
----
Here are some examples based on the following given case.
There are two HTable called test-01 and test-02, they have two column family cf1 and cf2 respectively, and deployed on the 3 regionservers.
see following table.
There are two HTable called test-01 and test-02, they have two column family cf1 and cf2 respectively, and deployed on the 3 RegionServers.
see following table.
[cols="1,1,1", options="header"]
|===
| RegionServer
| test-01
| test-02
|rs1| r1| r2
|rs2 |r2 |
|rs3 |r2 |r1
| rs1 | r1 | r2
| rs2 | r2 |
| rs3 | r2 | r1
|===
Following are some examples based on the previous given case.
Following are some examples based on the previous given case.
==== Canary test for every column family (store) of every region of every table
----
$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary
3/12/09 03:26:32 INFO tool.Canary: read from region test-01,,1386230156732.0e3c7d77ffb6361ea1b996ac1042ca9a. column family cf1 in 2ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-01,,1386230156732.0e3c7d77ffb6361ea1b996ac1042ca9a. column family cf2 in 2ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-01,0004883,1386230156732.87b55e03dfeade00f441125159f8ca87. column family cf1 in 4ms
@ -139,23 +138,23 @@ $ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary
----
So you can see, table test-01 has two regions and two column families, so the Canary tool will pick 4 small piece of data from 4 (2 region * 2 store) different stores.
This is a default behavior of the this tool does.
This is a default behavior of the this tool does.
==== Canary test for every column family (store) of every region of specifictable(s)
==== Canary test for every column family (store) of every region of specific table(s)
You can also test one or more specific tables.
----
$ ${HBASE_HOME}/bin/hbase orghapache.hadoop.hbase.tool.Canary test-01 test-02
$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary test-01 test-02
----
==== Canary test with regionserver granularity
==== Canary test with RegionServer granularity
This will pick one small piece of data from each regionserver, and can also put your resionserver name as input options for canary-test specific regionservers.
This will pick one small piece of data from each RegionServer, and can also put your RegionServer name as input options for canary-test specific RegionServer.
----
$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -regionserver
13/12/09 06:05:17 INFO tool.Canary: Read from table:test-01 on region server:rs2 in 72ms
13/12/09 06:05:17 INFO tool.Canary: Read from table:test-02 on region server:rs3 in 34ms
13/12/09 06:05:17 INFO tool.Canary: Read from table:test-01 on region server:rs1 in 56ms
@ -166,33 +165,32 @@ $ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -regionserver
This will test both table test-01 and test-02.
----
$ ${HBASE_HOME}/bin/hbase orghapache.hadoop.hbase.tool.Canary -e test-0[1-2]
$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -e test-0[1-2]
----
==== Run canary test as daemon mode
Run repeatedly with interval defined in option -interval whose default value is 6 seconds.
Run repeatedly with interval defined in option `-interval` whose default value is 6 seconds.
This daemon will stop itself and return non-zero error code if any error occurs, due to the default value of option -f is true.
----
$ ${HBASE_HOME}/bin/hbase orghapache.hadoop.hbase.tool.Canary -daemon
$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -daemon
----
Run repeatedly with internal 5 seconds and will not stop itself even error occurs in the test.
Run repeatedly with internal 5 seconds and will not stop itself even if errors occur in the test.
----
$ ${HBASE_HOME}/bin/hbase orghapache.hadoop.hbase.tool.Canary -daemon -interval 50000 -f false
$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -daemon -interval 50000 -f false
----
==== Force timeout if canary test stuck
In some cases, we suffered the request stucked on the regionserver and not response back to the client.
The regionserver in problem, would also not indicated to be dead by Master, which would bring the clients hung.
So we provide the timeout option to kill the canary test forcefully and return non-zero error code as well.
In some cases the request is stuck and no response is sent back to the client. This can happen with dead RegionServers which the master has not yet noticed.
Because of this we provide a timeout option to kill the canary test and return a non-zero error code.
This run sets the timeout value to 60 seconds, the default value is 600 seconds.
----
$ ${HBASE_HOME}/bin/hbase orghapache.hadoop.hbase.tool.Canary -t 600000
$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.tool.Canary -t 600000
----
==== Running Canary in a Kerberos-enabled Cluster
@ -215,7 +213,6 @@ This example shows each of the properties with valid values.
[source,xml]
----
<property>
<name>hbase.client.kerberos.principal</name>
<value>hbase/_HOST@YOUR-REALM.COM</value>
@ -239,14 +236,14 @@ property>
[[health.check]]
=== Health Checker
You can configure HBase to run a script on a period and if it fails N times (configurable), have the server exit.
See link:[HBASE-7351 Periodic health check script] for configurations and detail.
You can configure HBase to run a script periodically and if it fails N times (configurable), have the server exit.
See _HBASE-7351 Periodic health check script_ for configurations and detail.
=== Driver
Several frequently-accessed utilities are provided as `Driver` classes, and executed by the _bin/hbase_ command.
These utilities represent MapReduce jobs which run on your cluster.
They are run in the following way, replacing [replaceable]_UtilityName_ with the utility you want to run.
They are run in the following way, replacing _UtilityName_ with the utility you want to run.
This command assumes you have set the environment variable `HBASE_HOME` to the directory where HBase is unpacked on your server.
----
@ -256,45 +253,45 @@ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.mapreduce.UtilityName
The following utilities are available:
+LoadIncrementalHFiles+::
`LoadIncrementalHFiles`::
Complete a bulk data load.
+CopyTable+::
`CopyTable`::
Export a table from the local cluster to a peer cluster.
+Export+::
`Export`::
Write table data to HDFS.
+Import+::
Import data written by a previous +Export+ operation.
`Import`::
Import data written by a previous `Export` operation.
+ImportTsv+::
`ImportTsv`::
Import data in TSV format.
+RowCounter+::
`RowCounter`::
Count rows in an HBase table.
+replication.VerifyReplication+::
`replication.VerifyReplication`::
Compare the data from tables in two different clusters.
WARNING: It doesn't work for incrementColumnValues'd cells since the timestamp is changed.
Note that this command is in a different package than the others.
Each command except +RowCounter+ accepts a single `--help` argument to print usage instructions.
Each command except `RowCounter` accepts a single `--help` argument to print usage instructions.
[[hbck]]
=== HBase +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`.
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 latter is an experimental feature).
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).
For more information, see <<hbck.in.depth,hbck.in.depth>>.
For more information, see <<hbck.in.depth>>.
[[hfile_tool2]]
=== HFile Tool
See <<hfile_tool,hfile tool>>.
See <<hfile_tool>>.
=== WAL Tools
@ -311,7 +308,7 @@ You can get a textual dump of a WAL file content by doing the following:
$ ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.FSHLog --dump hdfs://example.org:8020/hbase/.logs/example.org,60020,1283516293161/10.10.21.10%3A60020.1283973724012
----
The return code will be non-zero if issues with the file so you can test wholesomeness of file by redirecting `STDOUT` to `/dev/null` and testing the program return.
The return code will be non-zero if there are any issues with the file so you can test wholesomeness of file by redirecting `STDOUT` to `/dev/null` and testing the program return.
Similarly you can force a split of a log file directory by doing:
@ -323,7 +320,7 @@ Similarly you can force a split of a log file directory by doing:
===== WAL Pretty Printer
The WAL Pretty Printer is a tool with configurable options to print the contents of a WAL.
You can invoke it via the hbase cli with the 'wal' command.
You can invoke it via the HBase cli with the 'wal' command.
----
$ ./bin/hbase wal hdfs://example.org:8020/hbase/.logs/example.org,60020,1283516293161/10.10.21.10%3A60020.1283973724012
@ -333,7 +330,7 @@ You can invoke it via the hbase cli with the 'wal' command.
[NOTE]
====
Prior to version 2.0, the WAL Pretty Printer was called the `HLogPrettyPrinter`, after an internal name for HBase's write ahead log.
In those versions, you can pring the contents of a WAL using the same configuration as above, but with the 'hlog' command.
In those versions, you can pring the contents of a WAL using the same configuration as above, but with the 'hlog' command.
----
$ ./bin/hbase hlog hdfs://example.org:8020/hbase/.logs/example.org,60020,1283516293161/10.10.21.10%3A60020.1283973724012
@ -353,12 +350,12 @@ The usage is as follows:
----
$ ./bin/hbase org.apache.hadoop.hbase.mapreduce.CopyTable --help
$ ./bin/hbase org.apache.hadoop.hbase.mapreduce.CopyTable --help
/bin/hbase org.apache.hadoop.hbase.mapreduce.CopyTable --help
Usage: CopyTable [general options] [--starttime=X] [--endtime=Y] [--new.name=NEW] [--peer.adr=ADR] <tablename>
Options:
rs.class hbase.regionserver.class of the peer cluster,
rs.class hbase.regionserver.class of the peer cluster,
specify if different from current cluster
rs.impl hbase.regionserver.impl of the peer cluster,
startrow the start row
@ -394,17 +391,17 @@ For performance consider the following general options:
.Scanner Caching
[NOTE]
====
Caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration.
Caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration.
====
.Versions
[NOTE]
====
By default, CopyTable utility only copies the latest version of row cells unless `--versions=n` is explicitly specified in the command.
By default, CopyTable utility only copies the latest version of row cells unless `--versions=n` is explicitly specified in the command.
====
See Jonathan Hsieh's link:http://www.cloudera.com/blog/2012/06/online-hbase-backups-with-copytable-2/[Online
HBase Backups with CopyTable] blog post for more on +CopyTable+.
HBase Backups with CopyTable] blog post for more on `CopyTable`.
=== Export
@ -415,7 +412,7 @@ Invoke via:
$ bin/hbase org.apache.hadoop.hbase.mapreduce.Export <tablename> <outputdir> [<versions> [<starttime> [<endtime>]]]
----
Note: caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration.
Note: caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration.
=== Import
@ -435,7 +432,7 @@ $ bin/hbase -Dhbase.import.version=0.94 org.apache.hadoop.hbase.mapreduce.Import
=== ImportTsv
ImportTsv is a utility that will load data in TSV format into HBase.
It has two distinct usages: loading data from TSV format in HDFS into HBase via Puts, and preparing StoreFiles to be loaded via the `completebulkload`.
It has two distinct usages: loading data from TSV format in HDFS into HBase via Puts, and preparing StoreFiles to be loaded via the `completebulkload`.
To load data via Puts (i.e., non-bulk loading):
@ -450,12 +447,12 @@ To generate StoreFiles for bulk-loading:
$ bin/hbase org.apache.hadoop.hbase.mapreduce.ImportTsv -Dimporttsv.columns=a,b,c -Dimporttsv.bulk.output=hdfs://storefile-outputdir <tablename> <hdfs-data-inputdir>
----
These generated StoreFiles can be loaded into HBase via <<completebulkload,completebulkload>>.
These generated StoreFiles can be loaded into HBase via <<completebulkload,completebulkload>>.
[[importtsv.options]]
==== ImportTsv Options
Running +ImportTsv+ with no arguments prints brief usage information:
Running `ImportTsv` with no arguments prints brief usage information:
----
@ -486,9 +483,9 @@ Other options that may be specified with -D include:
[[importtsv.example]]
==== ImportTsv Example
For example, assume that we are loading data into a table called 'datatsv' with a ColumnFamily called 'd' with two columns "c1" and "c2".
For example, assume that we are loading data into a table called 'datatsv' with a ColumnFamily called 'd' with two columns "c1" and "c2".
Assume that an input file exists as follows:
Assume that an input file exists as follows:
----
row1 c1 c2
@ -501,7 +498,7 @@ row7 c1 c2
row8 c1 c2
row9 c1 c2
row10 c1 c2
----
----
For ImportTsv to use this imput file, the command line needs to look like this:
@ -511,12 +508,12 @@ For ImportTsv to use this imput file, the command line needs to look like this:
----
\... and in this example the first column is the rowkey, which is why the HBASE_ROW_KEY is used.
The second and third columns in the file will be imported as "d:c1" and "d:c2", respectively.
The second and third columns in the file will be imported as "d:c1" and "d:c2", respectively.
[[importtsv.warning]]
==== ImportTsv Warning
If you have preparing a lot of data for bulk loading, make sure the target HBase table is pre-split appropriately.
If you have preparing a lot of data for bulk loading, make sure the target HBase table is pre-split appropriately.
[[importtsv.also]]
==== See Also
@ -526,7 +523,7 @@ For more information about bulk-loading HFiles into HBase, see <<arch.bulk.load,
=== CompleteBulkLoad
The `completebulkload` utility will move generated StoreFiles into an HBase table.
This utility is often used in conjunction with output from <<importtsv,importtsv>>.
This utility is often used in conjunction with output from <<importtsv,importtsv>>.
There are two ways to invoke this utility, with explicit classname and via the driver:
@ -546,16 +543,16 @@ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` ${HADOOP_HOME}/bin/hadoop j
Data generated via MapReduce is often created with file permissions that are not compatible with the running HBase process.
Assuming you're running HDFS with permissions enabled, those permissions will need to be updated before you run CompleteBulkLoad.
For more information about bulk-loading HFiles into HBase, see <<arch.bulk.load,arch.bulk.load>>.
For more information about bulk-loading HFiles into HBase, see <<arch.bulk.load,arch.bulk.load>>.
=== WALPlayer
WALPlayer is a utility to replay WAL files into HBase.
WALPlayer is a utility to replay WAL files into HBase.
The WAL can be replayed for a set of tables or all tables, and a timerange can be provided (in milliseconds). The WAL is filtered to this set of tables.
The output can optionally be mapped to another set of tables.
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.
WALPlayer can also generate HFiles for later bulk importing, in that case only a single table and no mapping can be specified.
Invoke via:
@ -570,7 +567,7 @@ $ bin/hbase org.apache.hadoop.hbase.mapreduce.WALPlayer /backuplogdir oldTable1,
----
WALPlayer, by default, runs as a mapreduce job.
To NOT run WALPlayer as a mapreduce job on your cluster, force it to run all in the local process by adding the flags `-Dmapreduce.jobtracker.address=local` on the command line.
To NOT run WALPlayer as a mapreduce job on your cluster, force it to run all in the local process by adding the flags `-Dmapreduce.jobtracker.address=local` on the command line.
[[rowcounter]]
=== RowCounter and CellCounter
@ -583,11 +580,11 @@ It will run the mapreduce all in a single process but it will run faster if you
$ bin/hbase org.apache.hadoop.hbase.mapreduce.RowCounter <tablename> [<column1> <column2>...]
----
Note: caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration.
Note: caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration.
HBase ships another diagnostic mapreduce job called link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/CellCounter.html[CellCounter].
Like RowCounter, it gathers more fine-grained statistics about your table.
The statistics gathered by RowCounter are more fine-grained and include:
The statistics gathered by RowCounter are more fine-grained and include:
* Total number of rows in the table.
* Total number of CFs across all rows.
@ -604,13 +601,13 @@ Use `hbase.mapreduce.scan.column.family` to specify scanning a single column fam
$ bin/hbase org.apache.hadoop.hbase.mapreduce.CellCounter <tablename> <outputDir> [regex or prefix]
----
Note: just like RowCounter, caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration.
Note: just like RowCounter, caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration.
=== mlockall
It is possible to optionally pin your servers in physical memory making them less likely to be swapped out in oversubscribed environments by having the servers call link:http://linux.die.net/man/2/mlockall[mlockall] on startup.
See link:https://issues.apache.org/jira/browse/HBASE-4391[HBASE-4391 Add ability to
start RS as root and call mlockall] for how to build the optional library and have it run on startup.
start RS as root and call mlockall] for how to build the optional library and have it run on startup.
[[compaction.tool]]
=== Offline Compaction Tool
@ -618,14 +615,14 @@ See link:https://issues.apache.org/jira/browse/HBASE-4391[HBASE-4391 Add ability
See the usage for the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/CompactionTool.html[Compaction
Tool].
Run it like this +./bin/hbase
org.apache.hadoop.hbase.regionserver.CompactionTool+
org.apache.hadoop.hbase.regionserver.CompactionTool+
=== +hbase clean+
=== `hbase clean`
The +hbase clean+ command cleans HBase data from ZooKeeper, HDFS, or both.
The `hbase clean` command cleans HBase data from ZooKeeper, HDFS, or both.
It is appropriate to use for testing.
Run it with no options for usage instructions.
The +hbase clean+ command was introduced in HBase 0.98.
The `hbase clean` command was introduced in HBase 0.98.
----
@ -637,25 +634,25 @@ Options:
--cleanAll cleans hbase related data from both zookeeper and hdfs.
----
=== +hbase pe+
=== `hbase pe`
The +hbase pe+ command is a shortcut provided to run the `org.apache.hadoop.hbase.PerformanceEvaluation` tool, which is used for testing.
The +hbase pe+ command was introduced in HBase 0.98.4.
The `hbase pe` command is a shortcut provided to run the `org.apache.hadoop.hbase.PerformanceEvaluation` tool, which is used for testing.
The `hbase pe` command was introduced in HBase 0.98.4.
The PerformanceEvaluation tool accepts many different options and commands.
For usage instructions, run the command with no options.
To run PerformanceEvaluation prior to HBase 0.98.4, issue the command +hbase org.apache.hadoop.hbase.PerformanceEvaluation+.
To run PerformanceEvaluation prior to HBase 0.98.4, issue the command `hbase org.apache.hadoop.hbase.PerformanceEvaluation`.
The PerformanceEvaluation tool has received many updates in recent HBase releases, including support for namespaces, support for tags, cell-level ACLs and visibility labels, multiget support for RPC calls, increased sampling sizes, an option to randomly sleep during testing, and ability to "warm up" the cluster before testing starts.
=== +hbase ltt+
=== `hbase ltt`
The +hbase ltt+ command is a shortcut provided to run the `org.apache.hadoop.hbase.util.LoadTestTool` utility, which is used for testing.
The +hbase ltt+ command was introduced in HBase 0.98.4.
The `hbase ltt` command is a shortcut provided to run the `org.apache.hadoop.hbase.util.LoadTestTool` utility, which is used for testing.
The `hbase ltt` command was introduced in HBase 0.98.4.
You must specify either +-write+ or +-update-read+ as the first option.
For general usage instructions, pass the +-h+ option.
You must specify either `-write` or `-update-read` as the first option.
For general usage instructions, pass the `-h` option.
To run LoadTestTool prior to HBase 0.98.4, issue the command +hbase
org.apache.hadoop.hbase.util.LoadTestTool+.
@ -668,10 +665,10 @@ The LoadTestTool has received many updates in recent HBase releases, including s
[[ops.regionmgt.majorcompact]]
=== Major Compaction
Major compactions can be requested via the HBase shell or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html#majorCompact%28java.lang.String%29[HBaseAdmin.majorCompact].
Major compactions can be requested via the HBase shell or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html#majorCompact%28java.lang.String%29[HBaseAdmin.majorCompact].
Note: major compactions do NOT do region merges.
See <<compaction,compaction>> for more information about compactions.
See <<compaction,compaction>> for more information about compactions.
[[ops.regionmgt.merge]]
=== Merge
@ -686,13 +683,13 @@ $ bin/hbase org.apache.hadoop.hbase.util.Merge <tablename> <region1> <region2>
If you feel you have too many regions and want to consolidate them, Merge is the utility you need.
Merge must run be done when the cluster is down.
See the link:http://ofps.oreilly.com/titles/9781449396107/performance.html[O'Reilly HBase
Book] for an example of usage.
Book] for an example of usage.
You will need to pass 3 parameters to this application.
The first one is the table name.
The second one is the fully qualified name of the first region to merge, like "table_name,\x0A,1342956111995.7cef47f192318ba7ccc75b1bbf27a82b.". The third one is the fully qualified name for the second region to merge.
The second one is the fully qualified name of the first region to merge, like "table_name,\x0A,1342956111995.7cef47f192318ba7ccc75b1bbf27a82b.". The third one is the fully qualified name for the second region to merge.
Additionally, there is a Ruby script attached to link:https://issues.apache.org/jira/browse/HBASE-1621[HBASE-1621] for region merging.
Additionally, there is a Ruby script attached to link:https://issues.apache.org/jira/browse/HBASE-1621[HBASE-1621] for region merging.
[[node.management]]
== Node Management
@ -708,14 +705,14 @@ $ ./bin/hbase-daemon.sh stop regionserver
The RegionServer will first close all regions and then shut itself down.
On shutdown, the RegionServer's ephemeral node in ZooKeeper will expire.
The master will notice the RegionServer gone and will treat it as a 'crashed' server; it will reassign the nodes the RegionServer was carrying.
The master will notice the RegionServer gone and will treat it as a 'crashed' server; it will reassign the nodes the RegionServer was carrying.
.Disable the Load Balancer before Decommissioning a node
[NOTE]
====
If the load balancer runs while a node is shutting down, then there could be contention between the Load Balancer and the Master's recovery of the just decommissioned RegionServer.
Avoid any problems by disabling the balancer first.
See <<lb,lb>> below.
See <<lb,lb>> below.
====
.Kill Node Tool
@ -726,7 +723,7 @@ Hardware issues could be detected by specialized monitoring tools before the zo
It deletes all the znodes of the server, starting the recovery process.
Plug in the script into your monitoring/fault detection tools to initiate faster failover.
Be careful how you use this disruptive tool.
Copy the script if you need to make use of it in a version of hbase previous to hbase-2.0.
Copy the script if you need to make use of it in a version of hbase previous to hbase-2.0.
====
A downside to the above stop of a RegionServer is that regions could be offline for a good period of time.
@ -748,7 +745,7 @@ Usage: graceful_stop.sh [--config &conf-dir>] [--restart] [--reload] [--thrift]
----
To decommission a loaded RegionServer, run the following: +$
./bin/graceful_stop.sh HOSTNAME+ where `HOSTNAME` is the host carrying the RegionServer you would decommission.
./bin/graceful_stop.sh HOSTNAME+ where `HOSTNAME` is the host carrying the RegionServer you would decommission.
.On `HOSTNAME`
[NOTE]
@ -757,18 +754,18 @@ The `HOSTNAME` passed to _graceful_stop.sh_ must match the hostname that hbase i
Check the list of RegionServers in the master UI for how HBase is referring to servers.
Its usually hostname but can also be FQDN.
Whatever HBase is using, this is what you should pass the _graceful_stop.sh_ decommission script.
If you pass IPs, the script is not yet smart enough to make a hostname (or FQDN) of it and so it will fail when it checks if server is currently running; the graceful unloading of regions will not run.
If you pass IPs, the script is not yet smart enough to make a hostname (or FQDN) of it and so it will fail when it checks if server is currently running; the graceful unloading of regions will not run.
====
The _graceful_stop.sh_ script will move the regions off the decommissioned RegionServer one at a time to minimize region churn.
It will verify the region deployed in the new location before it will moves the next region and so on until the decommissioned server is carrying zero regions.
At this point, the _graceful_stop.sh_ tells the RegionServer +stop+.
The master will at this point notice the RegionServer gone but all regions will have already been redeployed and because the RegionServer went down cleanly, there will be no WAL logs to split.
At this point, the _graceful_stop.sh_ tells the RegionServer `stop`.
The master will at this point notice the RegionServer gone but all regions will have already been redeployed and because the RegionServer went down cleanly, there will be no WAL logs to split.
.Load Balancer
[NOTE]
====
It is assumed that the Region Load Balancer is disabled while the +graceful_stop+ script runs (otherwise the balancer and the decommission script will end up fighting over region deployments). Use the shell to disable the balancer:
It is assumed that the Region Load Balancer is disabled while the `graceful_stop` script runs (otherwise the balancer and the decommission script will end up fighting over region deployments). Use the shell to disable the balancer:
[source]
----
@ -787,9 +784,9 @@ false
0 row(s) in 0.3590 seconds
----
The +graceful_stop+ will check the balancer and if enabled, will turn it off before it goes to work.
The `graceful_stop` will check the balancer and if enabled, will turn it off before it goes to work.
If it exits prematurely because of error, it will not have reset the balancer.
Hence, it is better to manage the balancer apart from +graceful_stop+ reenabling it after you are done w/ graceful_stop.
Hence, it is better to manage the balancer apart from `graceful_stop` reenabling it after you are done w/ graceful_stop.
====
[[draining.servers]]
@ -798,7 +795,7 @@ Hence, it is better to manage the balancer apart from +graceful_stop+ reenabling
If you have a large cluster, you may want to decommission more than one machine at a time by gracefully stopping mutiple RegionServers concurrently.
To gracefully drain multiple regionservers at the same time, RegionServers can be put into a "draining" state.
This is done by marking a RegionServer as a draining node by creating an entry in ZooKeeper under the _hbase_root/draining_ znode.
This znode has format `name,port,startcode` just like the regionserver entries under _hbase_root/rs_ znode.
This znode has format `name,port,startcode` just like the regionserver entries under _hbase_root/rs_ znode.
Without this facility, decommissioning mulitple nodes may be non-optimal because regions that are being drained from one region server may be moved to other regionservers that are also draining.
Marking RegionServers to be in the draining state prevents this from happening.
@ -814,7 +811,7 @@ take a while to go down spewing errors in _dmesg_ -- or for some reason, run muc
In this case you want to decommission the disk.
You have two options.
You can link:http://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
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.
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
[NOTE]
@ -833,7 +830,7 @@ See the release notes for release you want to upgrade to, to find out about limi
There are multiple ways to restart your cluster nodes, depending on your situation.
These methods are detailed below.
==== Using the +rolling-restart.sh+ Script
==== Using the `rolling-restart.sh` Script
HBase ships with a script, _bin/rolling-restart.sh_, that allows you to perform rolling restarts on the entire cluster, the master only, or the RegionServers only.
The script is provided as a template for your own script, and is not explicitly tested.
@ -869,7 +866,7 @@ Limiting the Number of Threads::
==== Manual Rolling Restart
To retain more control over the process, you may wish to manually do a rolling restart across your cluster.
This uses the +graceful-stop.sh+ command <<decommission,decommission>>.
This uses the `graceful-stop.sh` command <<decommission,decommission>>.
In this method, you can restart each RegionServer individually and then move its old regions back into place, retaining locality.
If you also need to restart the Master, you need to do it separately, and restart the Master before restarting the RegionServers using this method.
The following is an example of such a command.
@ -882,13 +879,13 @@ It disables the load balancer before moving the regions.
$ for i in `cat conf/regionservers|sort`; do ./bin/graceful_stop.sh --restart --reload --debug $i; done &> /tmp/log.txt &;
----
Monitor the output of the _/tmp/log.txt_ file to follow the progress of the script.
Monitor the output of the _/tmp/log.txt_ file to follow the progress of the script.
==== Logic for Crafting Your Own Rolling Restart Script
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.
. 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.
+
----
@ -915,12 +912,12 @@ $ for i in `cat conf/regionservers|sort`; do ./bin/graceful_stop.sh --restart --
----
+
If you are running Thrift or REST servers, pass the --thrift or --rest options.
For other available options, run the +bin/graceful-stop.sh --help+ command.
For other available options, run the `bin/graceful-stop.sh --help` command.
+
It is important to drain HBase regions slowly when restarting multiple RegionServers.
Otherwise, multiple regions go offline simultaneously and must be reassigned to other nodes, which may also go offline soon.
This can negatively affect performance.
You can inject delays into the script above, for instance, by adding a Shell command such as +sleep+.
You can inject delays into the script above, for instance, by adding a Shell command such as `sleep`.
To wait for 5 minutes between each RegionServer restart, modify the above script to the following:
+
----
@ -929,24 +926,24 @@ $ 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.
. Run the `hbck` utility again, to be sure the cluster is consistent.
[[adding.new.node]]
=== Adding a New Node
Adding a new regionserver in HBase is essentially free, you simply start it like this: +$ ./bin/hbase-daemon.sh start regionserver+ and it will register itself with the master.
Adding a new regionserver in HBase is essentially free, you simply start it like this: `$ ./bin/hbase-daemon.sh start regionserver` and it will register itself with the master.
Ideally you also started a DataNode on the same machine so that the RS can eventually start to have local files.
If you rely on ssh to start your daemons, don't forget to add the new hostname in _conf/regionservers_ on the master.
If you rely on ssh to start your daemons, don't forget to add the new hostname in _conf/regionservers_ on the master.
At this point the region server isn't serving data because no regions have moved to it yet.
If the balancer is enabled, it will start moving regions to the new RS.
On a small/medium cluster this can have a very adverse effect on latency as a lot of regions will be offline at the same time.
It is thus recommended to disable the balancer the same way it's done when decommissioning a node and move the regions manually (or even better, using a script that moves them one by one).
It is thus recommended to disable the balancer the same way it's done when decommissioning a node and move the regions manually (or even better, using a script that moves them one by one).
The moved regions will all have 0% locality and won't have any blocks in cache so the region server will have to use the network to serve requests.
Apart from resulting in higher latency, it may also be able to use all of your network card's capacity.
For practical purposes, consider that a standard 1GigE NIC won't be able to read much more than _100MB/s_.
In this case, or if you are in a OLAP environment and require having locality, then it is recommended to major compact the moved regions.
In this case, or if you are in a OLAP environment and require having locality, then it is recommended to major compact the moved regions.
== HBase Metrics
@ -965,7 +962,7 @@ To configure metrics for a given region server, edit the _conf/hadoop-metrics2-h
Restart the region server for the changes to take effect.
To change the sampling rate for the default sink, edit the line beginning with `*.period`.
To filter which metrics are emitted or to extend the metrics framework, see link:http://hadoop.apache.org/docs/current/api/org/apache/hadoop/metrics2/package-summary.html
To filter which metrics are emitted or to extend the metrics framework, see link:http://hadoop.apache.org/docs/current/api/org/apache/hadoop/metrics2/package-summary.html
.HBase Metrics and Ganglia
[NOTE]
@ -993,19 +990,19 @@ Different metrics are exposed for the Master process and each region server proc
The metrics for the region server are presented as a dump of the JMX bean in JSON format.
This will dump out all metrics names and their values.
To include metrics descriptions in the listing -- this can be useful when you are exploring what is available -- add a query string of `?description=true` so your URL becomes `http://REGIONSERVER_HOSTNAME:60030/jmx?description=true`.
Not all beans and attributes have descriptions.
Not all beans and attributes have descriptions.
. To view metrics for the Master, connect to the Master's web UI instead (defaults to `http://localhost:60010` or port 16010 in HBase 1.0+) and click its [label]#Metrics
Dump# link.
To include metrics descriptions in the listing -- this can be useful when you are exploring what is available -- add a query string of `?description=true` so your URL becomes `http://REGIONSERVER_HOSTNAME:60010/jmx?description=true`.
Not all beans and attributes have descriptions.
Not all beans and attributes have descriptions.
You can use many different tools to view JMX content by browsing MBeans.
This procedure uses +jvisualvm+, which is an application usually available in the JDK.
This procedure uses `jvisualvm`, which is an application usually available in the JDK.
.Procedure: Browse the JMX Output of Available Metrics
. Start HBase, if it is not already running.
. Run the command +jvisualvm+ command on a host with a GUI display.
. Run the command `jvisualvm` command on a host with a GUI display.
You can launch it from the command line or another method appropriate for your operating system.
. Be sure the [label]#VisualVM-MBeans# plugin is installed. Browse to *Tools -> Plugins*. Click [label]#Installed# and check whether the plugin is listed.
If not, click [label]#Available Plugins#, select it, and click btn:[Install].
@ -1014,8 +1011,8 @@ This procedure uses +jvisualvm+, which is an application usually available in th
A detailed view opens in the right-hand panel.
Click the [label]#MBeans# tab which appears as a tab in the top of the right-hand panel.
. To access the HBase metrics, navigate to the appropriate sub-bean:
.* Master:
.* RegionServer:
.* Master:
.* RegionServer:
. The name of each metric and its current value is displayed in the [label]#Attributes# tab.
For a view which includes more details, including the description of each attribute, click the [label]#Metadata# tab.
@ -1051,7 +1048,7 @@ hbase.master.ritCountOverThreshold::
The number of regions that have been in transition longer than a threshold time (default: 60 seconds)
hbase.master.ritOldestAge::
The age of the longest region in transition, in milliseconds
The age of the longest region in transition, in milliseconds
[[rs_metrics]]
=== Most Important RegionServer Metrics
@ -1148,7 +1145,7 @@ hbase.regionserver.mutationsWithoutWALCount ::
=== Overview
The following metrics are arguably the most important to monitor for each RegionServer for "macro monitoring", preferably with a system like link:http://opentsdb.net/[OpenTSDB].
If your cluster is having performance issues it's likely that you'll see something unusual with this group.
If your cluster is having performance issues it's likely that you'll see something unusual with this group.
HBase::
* See <<rs_metrics,rs metrics>>
@ -1160,7 +1157,7 @@ OS::
Java::
* GC
For more information on HBase metrics, see <<hbase_metrics,hbase metrics>>.
For more information on HBase metrics, see <<hbase_metrics,hbase metrics>>.
[[ops.slow.query]]
=== Slow Query Log
@ -1168,18 +1165,18 @@ For more information on HBase metrics, see <<hbase_metrics,hbase metrics>>.
The HBase slow query log consists of parseable JSON structures describing the properties of those client operations (Gets, Puts, Deletes, etc.) that either took too long to run, or produced too much output.
The thresholds for "too long to run" and "too much output" are configurable, as described below.
The output is produced inline in the main region server logs so that it is easy to discover further details from context with other logged events.
It is also prepended with identifying tags [constant]+(responseTooSlow)+, [constant]+(responseTooLarge)+, [constant]+(operationTooSlow)+, and [constant]+(operationTooLarge)+ in order to enable easy filtering with grep, in case the user desires to see only slow queries.
It is also prepended with identifying tags `(responseTooSlow)`, `(responseTooLarge)`, `(operationTooSlow)`, and `(operationTooLarge)` in order to enable easy filtering with grep, in case the user desires to see only slow queries.
==== Configuration
There are two configuration knobs that can be used to adjust the thresholds for when queries are logged.
There are two configuration knobs that can be used to adjust the thresholds for when queries are logged.
* `hbase.ipc.warn.response.time` Maximum number of milliseconds that a query can be run without being logged.
Defaults to 10000, or 10 seconds.
Can be set to -1 to disable logging by time.
Can be set to -1 to disable logging by time.
* `hbase.ipc.warn.response.size` Maximum byte size of response that a query can return without being logged.
Defaults to 100 megabytes.
Can be set to -1 to disable logging by size.
Can be set to -1 to disable logging by size.
==== Metrics
@ -1190,8 +1187,8 @@ The slow query log exposes to metrics to JMX.
==== Output
The output is tagged with operation e.g. [constant]+(operationTooSlow)+ if the call was a client operation, such as a Put, Get, or Delete, which we expose detailed fingerprint information for.
If not, it is tagged [constant]+(responseTooSlow)+ and still produces parseable JSON output, but with less verbose information solely regarding its duration and size in the RPC itself. [constant]+TooLarge+ is substituted for [constant]+TooSlow+ if the response size triggered the logging, with [constant]+TooLarge+ appearing even in the case that both size and duration triggered logging.
The output is tagged with operation e.g. `(operationTooSlow)` if the call was a client operation, such as a Put, Get, or Delete, which we expose detailed fingerprint information for.
If not, it is tagged `(responseTooSlow)` and still produces parseable JSON output, but with less verbose information solely regarding its duration and size in the RPC itself. `TooLarge` is substituted for `TooSlow` if the response size triggered the logging, with `TooLarge` appearing even in the case that both size and duration triggered logging.
==== Example
@ -1199,13 +1196,13 @@ If not, it is tagged [constant]+(responseTooSlow)+ and still produces p
[source]
----
2011-09-08 10:01:25,824 WARN org.apache.hadoop.ipc.HBaseServer: (operationTooSlow): {"tables":{"riley2":{"puts":[{"totalColumns":11,"families":{"actions":[{"timestamp":1315501284459,"qualifier":"0","vlen":9667580},{"timestamp":1315501284459,"qualifier":"1","vlen":10122412},{"timestamp":1315501284459,"qualifier":"2","vlen":11104617},{"timestamp":1315501284459,"qualifier":"3","vlen":13430635}]},"row":"cfcd208495d565ef66e7dff9f98764da:0"}],"families":["actions"]}},"processingtimems":956,"client":"10.47.34.63:33623","starttimems":1315501284456,"queuetimems":0,"totalPuts":1,"class":"HRegionServer","responsesize":0,"method":"multiPut"}
----
----
Note that everything inside the "tables" structure is output produced by MultiPut's fingerprint, while the rest of the information is RPC-specific, such as processing time and client IP/port.
Other client operations follow the same pattern and the same general structure, with necessary differences due to the nature of the individual operations.
In the case that the call is not a client operation, that detailed fingerprint information will be completely absent.
In the case that the call is not a client operation, that detailed fingerprint information will be completely absent.
This particular example, for example, would indicate that the likely cause of slowness is simply a very large (on the order of 100MB) multiput, as we can tell by the "vlen," or value length, fields of each put in the multiPut.
This particular example, for example, would indicate that the likely cause of slowness is simply a very large (on the order of 100MB) multiput, as we can tell by the "vlen," or value length, fields of each put in the multiPut.
=== Block Cache Monitoring
@ -1230,7 +1227,7 @@ Have a look in the Web UI.
== Cluster Replication
NOTE: This information was previously available at link:http://hbase.apache.org/replication.html[Cluster Replication].
NOTE: This information was previously available at link:http://hbase.apache.org/replication.html[Cluster Replication].
HBase provides a cluster replication mechanism which allows you to keep one cluster's state synchronized with that of another cluster, using the write-ahead log (WAL) of the source cluster to propagate the changes.
Some use cases for cluster replication include:
@ -1282,7 +1279,7 @@ Use the arrows to follow the data paths.
image::hbase_replication_diagram.jpg[]
HBase replication borrows many concepts from the [firstterm]_statement-based replication_ design used by MySQL.
Instead of SQL statements, entire WALEdits (consisting of multiple cell inserts coming from Put and Delete operations on the clients) are replicated in order to maintain atomicity.
Instead of SQL statements, entire WALEdits (consisting of multiple cell inserts coming from Put and Delete operations on the clients) are replicated in order to maintain atomicity.
=== Configuring Cluster Replication
@ -1312,8 +1309,8 @@ If both clusters use the same ZooKeeper cluster, you must use a different `zooke
. On the source cluster, configure each column family to be replicated by setting its REPLICATION_SCOPE to 1, using commands such as the following in HBase Shell.
+
----
hbase> disable 'example_table'
hbase> alter 'example_table', {NAME => 'example_family', REPLICATION_SCOPE => '1'}
hbase> disable 'example_table'
hbase> alter 'example_table', {NAME => 'example_family', REPLICATION_SCOPE => '1'}
hbase> enable 'example_table'
----
@ -1321,7 +1318,7 @@ hbase> enable 'example_table'
+
----
Considering 1 rs, with ratio 0.1
Getting 1 rs from peer cluster # 0
Getting 1 rs from peer cluster # 0
Choosing peer 10.10.1.49:62020
----
@ -1334,7 +1331,7 @@ The command has the following form:
hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication [--starttime=timestamp1] [--stoptime=timestamp [--families=comma separated list of families] <peerId><tablename>
----
+
The `VerifyReplication` command prints out `GOODROWS` and `BADROWS` counters to indicate rows that did and did not replicate correctly.
The `VerifyReplication` command prints out `GOODROWS` and `BADROWS` counters to indicate rows that did and did not replicate correctly.
=== Detailed Information About Cluster Replication
@ -1613,10 +1610,10 @@ The following metrics are exposed at the global region server level and (since H
== HBase Backup
There are two broad strategies for performing HBase backups: backing up with a full cluster shutdown, and backing up on a live cluster.
Each approach has pros and cons.
Each approach has pros and cons.
For additional information, see link:http://blog.sematext.com/2011/03/11/hbase-backup-options/[HBase Backup
Options] over on the Sematext Blog.
Options] over on the Sematext Blog.
[[ops.backup.fullshutdown]]
=== Full Shutdown Backup
@ -1624,7 +1621,7 @@ For additional information, see link:http://blog.sematext.com/2011/03/11/hbase-b
Some environments can tolerate a periodic full shutdown of their HBase cluster, for example if it is being used a back-end analytic capacity and not serving front-end web-pages.
The benefits are that the NameNode/Master are RegionServers are down, so there is no chance of missing any in-flight changes to either StoreFiles or metadata.
The obvious con is that the cluster is down.
The steps include:
The steps include:
[[ops.backup.fullshutdown.stop]]
==== Stop HBase
@ -1634,47 +1631,47 @@ The steps include:
[[ops.backup.fullshutdown.distcp]]
==== Distcp
Distcp could be used to either copy the contents of the HBase directory in HDFS to either the same cluster in another directory, or to a different cluster.
Distcp could be used to either copy the contents of the HBase directory in HDFS to either the same cluster in another directory, or to a different cluster.
Note: Distcp works in this situation because the cluster is down and there are no in-flight edits to files.
Distcp-ing of files in the HBase directory is not generally recommended on a live cluster.
Distcp-ing of files in the HBase directory is not generally recommended on a live cluster.
[[ops.backup.fullshutdown.restore]]
==== Restore (if needed)
The backup of the hbase directory from HDFS is copied onto the 'real' hbase directory via distcp.
The act of copying these files creates new HDFS metadata, which is why a restore of the NameNode edits from the time of the HBase backup isn't required for this kind of restore, because it's a restore (via distcp) of a specific HDFS directory (i.e., the HBase part) not the entire HDFS file-system.
The act of copying these files creates new HDFS metadata, which is why a restore of the NameNode edits from the time of the HBase backup isn't required for this kind of restore, because it's a restore (via distcp) of a specific HDFS directory (i.e., the HBase part) not the entire HDFS file-system.
[[ops.backup.live.replication]]
=== Live Cluster Backup - Replication
This approach assumes that there is a second cluster.
See the HBase page on link:http://hbase.apache.org/replication.html[replication] for more information.
See the HBase page on link:http://hbase.apache.org/replication.html[replication] for more information.
[[ops.backup.live.copytable]]
=== Live Cluster Backup - CopyTable
The <<copytable,copytable>> utility could either be used to copy data from one table to another on the same cluster, or to copy data to another table on another cluster.
The <<copytable,copytable>> utility could either be used to copy data from one table to another on the same cluster, or to copy data to another table on another cluster.
Since the cluster is up, there is a risk that edits could be missed in the copy process.
Since the cluster is up, there is a risk that edits could be missed in the copy process.
[[ops.backup.live.export]]
=== Live Cluster Backup - Export
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.
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.
[[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.
Also, Exporting the snapshot to another cluster doesn't have impact on the Region Servers.
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.
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.
[[ops.snapshots.configuration]]
=== Configuration
@ -1707,7 +1704,7 @@ hbase> snapshot 'myTable', 'myTableSnapshot-122112'
The default behavior is to perform a flush of data in memory before the snapshot is taken.
This means that data in memory is included in the snapshot.
In most cases, this is the desired behavior.
However, if your set-up can tolerate data in memory being excluded from the snapshot, you can use the +SKIP_FLUSH+ option of the +snapshot+ command to disable and flushing while taking the snapshot.
However, if your set-up can tolerate data in memory being excluded from the snapshot, you can use the `SKIP_FLUSH` option of the `snapshot` command to disable and flushing while taking the snapshot.
----
hbase> snapshot 'mytable', 'snapshot123', {SKIP_FLUSH => true}
@ -1765,9 +1762,9 @@ hbase> restore_snapshot 'myTableSnapshot-122112'
----
NOTE: Since Replication works at log level and snapshots at file-system level, after a restore, the replicas will be in a different state from the master.
If you want to use restore, you need to stop replication and redo the bootstrap.
If you want to use restore, you need to stop replication and redo the bootstrap.
In case of partial data-loss due to misbehaving client, instead of a full restore that requires the table to be disabled, you can clone the table from the snapshot and use a Map-Reduce job to copy the data that you need, from the clone to the main one.
In case of partial data-loss due to misbehaving client, instead of a full restore that requires the table to be disabled, you can clone the table from the snapshot and use a Map-Reduce job to copy the data that you need, from the clone to the main one.
[[ops.snapshots.acls]]
=== Snapshots operations and ACLs
@ -1809,7 +1806,7 @@ Start with a solid understanding of how HBase handles data internally.
[[ops.capacity.nodes.datasize]]
==== Physical data size
Physical data size on disk is distinct from logical size of your data and is affected by the following:
Physical data size on disk is distinct from logical size of your data and is affected by the following:
* Increased by HBase overhead
+
@ -1868,7 +1865,7 @@ 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.
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]]
@ -1979,8 +1976,8 @@ For pre-splitting howto, see <<manual_region_splitting_decisions,manual region s
== Table Rename
In versions 0.90.x of hbase and earlier, we had a simple script that would rename the hdfs table directory and then do an edit of the hbase:meta table replacing all mentions of the old table name with the new.
The script was called +./bin/rename_table.rb+.
The script was deprecated and removed mostly because it was unmaintained and the operation performed by the script was brutal.
The script was called `./bin/rename_table.rb`.
The script was deprecated and removed mostly because it was unmaintained and the operation performed by the script was brutal.
As of hbase 0.94.x, you can use the snapshot facility renaming a table.
Here is how you would do it using the hbase shell:

View File

@ -31,9 +31,8 @@
.Apache HBase Orca
image::jumping-orca_rotated_25percent.png[]
link:https://issues.apache.org/jira/browse/HBASE-4920[An Orca is the Apache
HBase mascot.] See NOTICES.txt.
link:https://issues.apache.org/jira/browse/HBASE-4920[An Orca is the Apache HBase mascot.] See NOTICES.txt.
Our Orca logo we got here: http://www.vectorfree.com/jumping-orca It is licensed Creative Commons Attribution 3.0.
See https://creativecommons.org/licenses/by/3.0/us/ We changed the logo by stripping the colored background, inverting it and then rotating it some.
See https://creativecommons.org/licenses/by/3.0/us/ We changed the logo by stripping the colored background, inverting it and then rotating it some.
:numbered:

View File

@ -46,24 +46,24 @@ Use a 64-bit platform (and 64-bit JVM).
=== Swapping
Watch out for swapping.
Set swappiness to 0.
Set `swappiness` to 0.
[[perf.network]]
== Network
Perhaps the most important factor in avoiding network issues degrading Hadoop and HBase performance is the switching hardware that is used, decisions made early in the scope of the project can cause major problems when you double or triple the size of your cluster (or more).
Perhaps the most important factor in avoiding network issues degrading Hadoop and HBase performance is the switching hardware that is used, decisions made early in the scope of the project can cause major problems when you double or triple the size of your cluster (or more).
Important items to consider:
Important items to consider:
* Switching capacity of the device
* Number of systems connected
* Uplink capacity
* Uplink capacity
[[perf.network.1switch]]
=== Single Switch
The single most important factor in this configuration is that the switching capacity of the hardware is capable of handling the traffic which can be generated by all systems connected to the switch.
Some lower priced commodity hardware can have a slower switching capacity than could be utilized by a full switch.
Some lower priced commodity hardware can have a slower switching capacity than could be utilized by a full switch.
[[perf.network.2switch]]
=== Multiple Switches
@ -71,9 +71,9 @@ Some lower priced commodity hardware can have a slower switching capacity than c
Multiple switches are a potential pitfall in the architecture.
The most common configuration of lower priced hardware is a simple 1Gbps uplink from one switch to another.
This often overlooked pinch point can easily become a bottleneck for cluster communication.
Especially with MapReduce jobs that are both reading and writing a lot of data the communication across this uplink could be saturated.
Especially with MapReduce jobs that are both reading and writing a lot of data the communication across this uplink could be saturated.
Mitigation of this issue is fairly simple and can be accomplished in multiple ways:
Mitigation of this issue is fairly simple and can be accomplished in multiple ways:
* Use appropriate hardware for the scale of the cluster which you're attempting to build.
* Use larger single switch configurations i.e.
@ -83,7 +83,7 @@ Mitigation of this issue is fairly simple and can be accomplished in multiple wa
[[perf.network.multirack]]
=== Multiple Racks
Multiple rack configurations carry the same potential issues as multiple switches, and can suffer performance degradation from two main areas:
Multiple rack configurations carry the same potential issues as multiple switches, and can suffer performance degradation from two main areas:
* Poor switch capacity performance
* Insufficient uplink to another rack
@ -91,14 +91,14 @@ Multiple rack configurations carry the same potential issues as multiple switche
If the the switches in your rack have appropriate switching capacity to handle all the hosts at full speed, the next most likely issue will be caused by homing more of your cluster across racks.
The easiest way to avoid issues when spanning multiple racks is to use port trunking to create a bonded uplink to other racks.
The downside of this method however, is in the overhead of ports that could potentially be used.
An example of this is, creating an 8Gbps port channel from rack A to rack B, using 8 of your 24 ports to communicate between racks gives you a poor ROI, using too few however can mean you're not getting the most out of your cluster.
An example of this is, creating an 8Gbps port channel from rack A to rack B, using 8 of your 24 ports to communicate between racks gives you a poor ROI, using too few however can mean you're not getting the most out of your cluster.
Using 10Gbe links between racks will greatly increase performance, and assuming your switches support a 10Gbe uplink or allow for an expansion card will allow you to save your ports for machines as opposed to uplinks.
Using 10Gbe links between racks will greatly increase performance, and assuming your switches support a 10Gbe uplink or allow for an expansion card will allow you to save your ports for machines as opposed to uplinks.
[[perf.network.ints]]
=== Network Interfaces
Are all the network interfaces functioning correctly? Are you sure? See the Troubleshooting Case Study in <<casestudies.slownode,casestudies.slownode>>.
Are all the network interfaces functioning correctly? Are you sure? See the Troubleshooting Case Study in <<casestudies.slownode>>.
[[jvm]]
== Java
@ -109,35 +109,33 @@ Are all the network interfaces functioning correctly? Are you sure? See the Trou
[[gcpause]]
==== Long GC pauses
In his presentation, link:http://www.slideshare.net/cloudera/hbase-hug-presentation[Avoiding Full GCs
with MemStore-Local Allocation Buffers], Todd Lipcon describes two cases of stop-the-world garbage collections common in HBase, especially during loading; CMS failure modes and old generation heap fragmentation brought.
In his presentation, link:http://www.slideshare.net/cloudera/hbase-hug-presentation[Avoiding Full GCs with MemStore-Local Allocation Buffers], Todd Lipcon describes two cases of stop-the-world garbage collections common in HBase, especially during loading; CMS failure modes and old generation heap fragmentation brought.
To address the first, start the CMS earlier than default by adding `-XX:CMSInitiatingOccupancyFraction` and setting it down from defaults.
Start at 60 or 70 percent (The lower you bring down the threshold, the more GCing is done, the more CPU used). To address the second fragmentation issue, Todd added an experimental facility,
(((MSLAB))), that must be explicitly enabled in Apache HBase 0.90.x (Its defaulted to be on in Apache 0.92.x HBase). See `hbase.hregion.memstore.mslab.enabled` to true in your `Configuration`.
Start at 60 or 70 percent (The lower you bring down the threshold, the more GCing is done, the more CPU used). To address the second fragmentation issue, Todd added an experimental facility,
(MSLAB), that must be explicitly enabled in Apache HBase 0.90.x (It's defaulted to be _on_ in Apache 0.92.x HBase). Set `hbase.hregion.memstore.mslab.enabled` to true in your `Configuration`.
See the cited slides for background and detail.
The latest jvms do better regards fragmentation so make sure you are running a recent release.
Read down in the message, link:http://osdir.com/ml/hotspot-gc-use/2011-11/msg00002.html[Identifying
concurrent mode failures caused by fragmentation].
The latest JVMs do better regards fragmentation so make sure you are running a recent release.
Read down in the message, link:http://osdir.com/ml/hotspot-gc-use/2011-11/msg00002.html[Identifying concurrent mode failures caused by fragmentation].
Be aware that when enabled, each MemStore instance will occupy at least an MSLAB instance of memory.
If you have thousands of regions or lots of regions each with many column families, this allocation of MSLAB may be responsible for a good portion of your heap allocation and in an extreme case cause you to OOME.
Disable MSLAB in this case, or lower the amount of memory it uses or float less regions per server.
Disable MSLAB in this case, or lower the amount of memory it uses or float less regions per server.
If you have a write-heavy workload, check out link:https://issues.apache.org/jira/browse/HBASE-8163[HBASE-8163
MemStoreChunkPool: An improvement for JAVA GC when using MSLAB].
If you have a write-heavy workload, check out link:https://issues.apache.org/jira/browse/HBASE-8163[HBASE-8163 MemStoreChunkPool: An improvement for JAVA GC when using MSLAB].
It describes configurations to lower the amount of young GC during write-heavy loadings.
If you do not have HBASE-8163 installed, and you are trying to improve your young GC times, one trick to consider -- courtesy of our Liang Xie -- is to set the GC config `-XX:PretenureSizeThreshold` in _hbase-env.sh_ to be just smaller than the size of `hbase.hregion.memstore.mslab.chunksize` so MSLAB allocations happen in the tenured space directly rather than first in the young gen.
You'd do this because these MSLAB allocations are going to likely make it to the old gen anyways and rather than pay the price of a copies between s0 and s1 in eden space followed by the copy up from young to old gen after the MSLABs have achieved sufficient tenure, save a bit of YGC churn and allocate in the old gen directly.
You'd do this because these MSLAB allocations are going to likely make it to the old gen anyways and rather than pay the price of a copies between s0 and s1 in eden space followed by the copy up from young to old gen after the MSLABs have achieved sufficient tenure, save a bit of YGC churn and allocate in the old gen directly.
For more information about GC logs, see <<trouble.log.gc,trouble.log.gc>>.
For more information about GC logs, see <<trouble.log.gc>>.
Consider also enabling the offheap Block Cache.
Consider also enabling the off-heap Block Cache.
This has been shown to mitigate GC pause times.
See <<block.cache,block.cache>>
See <<block.cache>>
[[perf.configurations]]
== HBase Configurations
See <<recommended_configurations,recommended configurations>>.
See <<recommended_configurations>>.
[[perf.compactions.and.splits]]
=== Managing Compactions
@ -147,22 +145,22 @@ For larger systems, managing link:[compactions and splits] may be something you
[[perf.handlers]]
=== `hbase.regionserver.handler.count`
See <<hbase.regionserver.handler.count,hbase.regionserver.handler.count>>.
See <<hbase.regionserver.handler.count>>.
[[perf.hfile.block.cache.size]]
=== `hfile.block.cache.size`
See <<hfile.block.cache.size,hfile.block.cache.size>>.
A memory setting for the RegionServer process.
See <<hfile.block.cache.size>>.
A memory setting for the RegionServer process.
[[blockcache.prefetch]]
=== Prefetch Option for Blockcache
link:https://issues.apache.org/jira/browse/HBASE-9857[HBASE-9857] adds a new option to prefetch HFile contents when opening the blockcache, if a columnfamily or regionserver property is set.
link:https://issues.apache.org/jira/browse/HBASE-9857[HBASE-9857] adds a new option to prefetch HFile contents when opening the BlockCache, if a Column family or RegionServer property is set.
This option is available for HBase 0.98.3 and later.
The purpose is to warm the blockcache as rapidly as possible after the cache is opened, using in-memory table data, and not counting the prefetching as cache misses.
This is great for fast reads, but is not a good idea if the data to be preloaded will not fit into the blockcache.
It is useful for tuning the IO impact of prefetching versus the time before all data blocks are in cache.
The purpose is to warm the BlockCache as rapidly as possible after the cache is opened, using in-memory table data, and not counting the prefetching as cache misses.
This is great for fast reads, but is not a good idea if the data to be preloaded will not fit into the BlockCache.
It is useful for tuning the IO impact of prefetching versus the time before all data blocks are in cache.
To enable prefetching on a given column family, you can use HBase Shell or use the API.
@ -192,73 +190,73 @@ See the API documentation for link:https://hbase.apache.org/apidocs/org/apache/h
[[perf.rs.memstore.size]]
=== `hbase.regionserver.global.memstore.size`
See <<hbase.regionserver.global.memstore.size,hbase.regionserver.global.memstore.size>>.
This memory setting is often adjusted for the RegionServer process depending on needs.
See <<hbase.regionserver.global.memstore.size>>.
This memory setting is often adjusted for the RegionServer process depending on needs.
[[perf.rs.memstore.size.lower.limit]]
=== `hbase.regionserver.global.memstore.size.lower.limit`
See <<hbase.regionserver.global.memstore.size.lower.limit,hbase.regionserver.global.memstore.size.lower.limit>>.
This memory setting is often adjusted for the RegionServer process depending on needs.
See <<hbase.regionserver.global.memstore.size.lower.limit>>.
This memory setting is often adjusted for the RegionServer process depending on needs.
[[perf.hstore.blockingstorefiles]]
=== `hbase.hstore.blockingStoreFiles`
See <<hbase.hstore.blockingstorefiles,hbase.hstore.blockingStoreFiles>>.
If there is blocking in the RegionServer logs, increasing this can help.
See <<hbase.hstore.blockingstorefiles>>.
If there is blocking in the RegionServer logs, increasing this can help.
[[perf.hregion.memstore.block.multiplier]]
=== `hbase.hregion.memstore.block.multiplier`
See <<hbase.hregion.memstore.block.multiplier,hbase.hregion.memstore.block.multiplier>>.
If there is enough RAM, increasing this can help.
See <<hbase.hregion.memstore.block.multiplier>>.
If there is enough RAM, increasing this can help.
[[hbase.regionserver.checksum.verify.performance]]
=== `hbase.regionserver.checksum.verify`
Have HBase write the checksum into the datablock and save having to do the checksum seek whenever you read.
See <<hbase.regionserver.checksum.verify,hbase.regionserver.checksum.verify>>, <<hbase.hstore.bytes.per.checksum,hbase.hstore.bytes.per.checksum>> and <<hbase.hstore.checksum.algorithm,hbase.hstore.checksum.algorithm>> For more information see the release note on link:https://issues.apache.org/jira/browse/HBASE-5074[HBASE-5074 support checksums in HBase block cache].
See <<hbase.regionserver.checksum.verify>>, <<hbase.hstore.bytes.per.checksum>> and <<hbase.hstore.checksum.algorithm>>. For more information see the release note on link:https://issues.apache.org/jira/browse/HBASE-5074[HBASE-5074 support checksums in HBase block cache].
=== Tuning `callQueue` Options
link:https://issues.apache.org/jira/browse/HBASE-11355[HBASE-11355] introduces several callQueue tuning mechanisms which can increase performance.
link:https://issues.apache.org/jira/browse/HBASE-11355[HBASE-11355] introduces several callQueue tuning mechanisms which can increase performance.
See the JIRA for some benchmarking information.
* To increase the number of callqueues, set +hbase.ipc.server.num.callqueue+ to a value greater than `1`.
* To split the callqueue into separate read and write queues, set `hbase.ipc.server.callqueue.read.ratio` to a value between `0` and `1`.
This factor weights the queues toward writes (if below .5) or reads (if above .5). Another way to say this is that the factor determines what percentage of the split queues are used for reads.
The following examples illustrate some of the possibilities.
Note that you always have at least one write queue, no matter what setting you use.
+
To increase the number of callqueues, set `hbase.ipc.server.num.callqueue` to a value greater than `1`.
To split the callqueue into separate read and write queues, set `hbase.ipc.server.callqueue.read.ratio` to a value between `0` and `1`.
This factor weights the queues toward writes (if below .5) or reads (if above .5). Another way to say this is that the factor determines what percentage of the split queues are used for reads.
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.
Given a value of `10` for +hbase.ipc.server.num.callqueue+, 3 queues would be used for reads and 7 for writes.
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.
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.
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 `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.
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.
You can also split the read queues so that separate queues are used for short reads (from Get operations) and long reads (from Scan operations), by setting the `hbase.ipc.server.callqueue.scan.ratio` option.
This option is a factor between 0 and 1, which determine the ratio of read queues used for Gets and Scans.
More queues are used for Gets if the value is below `.5` and more are used for scans if the value is above `.5`.
No matter what setting you use, at least one read queue is used for Get operations.
* You can also split the read queues so that separate queues are used for short reads (from Get operations) and long reads (from Scan operations), by setting the +hbase.ipc.server.callqueue.scan.ratio+ option.
This option is a factor between 0 and 1, which determine the ratio of read queues used for Gets and Scans.
More queues are used for Gets if the value is below `.5` and more are used for scans if the value is above `.5`.
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.
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.
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.
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.
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.
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.
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.
You can use the new option `hbase.ipc.server.callqueue.handler.factor` to programmatically tune the number of queues:
* You can use the new option `hbase.ipc.server.callqueue.handler.factor` to programmatically tune the number of queues:
+
* A value of `0` uses a single shared queue between all the handlers.
* A value of `1` uses a separate queue for each handler.
* A value between `0` and `1` tunes the number of queues against the number of handlers.
@ -268,13 +266,13 @@ Having more queues, such as in a situation where you have one queue per handler,
The trade-off is that if you have some queues with long-running tasks, a handler may end up waiting to execute from that queue rather than processing another queue which has waiting tasks.
For these values to take effect on a given Region Server, the Region Server must be restarted.
For these values to take effect on a given RegionServer, the RegionServer must be restarted.
These parameters are intended for testing purposes and should be used carefully.
[[perf.zookeeper]]
== ZooKeeper
See <<zookeeper,zookeeper>> for information on configuring ZooKeeper, and see the part about having a dedicated disk.
See <<zookeeper>> for information on configuring ZooKeeper, and see the part about having a dedicated disk.
[[perf.schema]]
== Schema Design
@ -282,20 +280,20 @@ See <<zookeeper,zookeeper>> for information on configuring ZooKeeper, and see th
[[perf.number.of.cfs]]
=== Number of Column Families
See <<number.of.cfs,number.of.cfs>>.
See <<number.of.cfs>>.
[[perf.schema.keys]]
=== Key and Attribute Lengths
See <<keysize,keysize>>.
See also <<perf.compression.however,perf.compression.however>> for compression caveats.
See <<keysize>>.
See also <<perf.compression.however>> for compression caveats.
[[schema.regionsize]]
=== Table RegionSize
The regionsize can be set on a per-table basis via `setFileSize` on link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HTableDescriptor.html[HTableDescriptor] in the event where certain tables require different regionsizes than the configured default regionsize.
The regionsize can be set on a per-table basis via `setFileSize` on link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HTableDescriptor.html[HTableDescriptor] in the event where certain tables require different regionsizes than the configured default regionsize.
See <<ops.capacity.regions,ops.capacity.regions>> for more information.
See <<ops.capacity.regions>> for more information.
[[schema.bloom]]
=== Bloom Filters
@ -303,13 +301,13 @@ See <<ops.capacity.regions,ops.capacity.regions>> for more information.
A Bloom filter, named for its creator, Burton Howard Bloom, is a data structure which is designed to predict whether a given element is a member of a set of data.
A positive result from a Bloom filter is not always accurate, but a negative result is guaranteed to be accurate.
Bloom filters are designed to be "accurate enough" for sets of data which are so large that conventional hashing mechanisms would be impractical.
For more information about Bloom filters in general, refer to link:http://en.wikipedia.org/wiki/Bloom_filter.
For more information about Bloom filters in general, refer to http://en.wikipedia.org/wiki/Bloom_filter.
In terms of HBase, Bloom filters provide a lightweight in-memory structure to reduce the number of disk reads for a given Get operation (Bloom filters do not work with Scans) to only the StoreFiles likely to contain the desired Row.
The potential performance gain increases with the number of parallel reads.
The potential performance gain increases with the number of parallel reads.
The Bloom filters themselves are stored in the metadata of each HFile and never need to be updated.
When an HFile is opened because a region is deployed to a RegionServer, the Bloom filter is loaded into memory.
When an HFile is opened because a region is deployed to a RegionServer, the Bloom filter is loaded into memory.
HBase includes some tuning mechanisms for folding the Bloom filter to reduce the size and keep the false positive rate within a desired range.
@ -317,8 +315,7 @@ Bloom filters were introduced in link:https://issues.apache.org/jira/browse/HBAS
Since HBase 0.96, row-based Bloom filters are enabled by default.
(link:https://issues.apache.org/jira/browse/HBASE-8450[HBASE-])
For more information on Bloom filters in relation to HBase, see <<blooms,blooms>> for more information, or the following Quora discussion: link:http://www.quora.com/How-are-bloom-filters-used-in-HBase[How are bloom
filters used in HBase?].
For more information on Bloom filters in relation to HBase, see <<blooms>> for more information, or the following Quora discussion: link:http://www.quora.com/How-are-bloom-filters-used-in-HBase[How are bloom filters used in HBase?].
[[bloom.filters.when]]
==== When To Use Bloom Filters
@ -327,16 +324,16 @@ Since HBase 0.96, row-based Bloom filters are enabled by default.
You may choose to disable them or to change some tables to use row+column Bloom filters, depending on the characteristics of your data and how it is loaded into HBase.
To determine whether Bloom filters could have a positive impact, check the value of `blockCacheHitRatio` in the RegionServer metrics.
If Bloom filters are enabled, the value of `blockCacheHitRatio` should increase, because the Bloom filter is filtering out blocks that are definitely not needed.
If Bloom filters are enabled, the value of `blockCacheHitRatio` should increase, because the Bloom filter is filtering out blocks that are definitely not needed.
You can choose to enable Bloom filters for a row or for a row+column combination.
If you generally scan entire rows, the row+column combination will not provide any benefit.
A row-based Bloom filter can operate on a row+column Get, but not the other way around.
However, if you have a large number of column-level Puts, such that a row may be present in every StoreFile, a row-based filter will always return a positive result and provide no benefit.
Unless you have one column per row, row+column Bloom filters require more space, in order to store more keys.
Bloom filters work best when the size of each data entry is at least a few kilobytes in size.
Bloom filters work best when the size of each data entry is at least a few kilobytes in size.
Overhead will be reduced when your data is stored in a few larger StoreFiles, to avoid extra disk IO during low-level scans to find a specific row.
Overhead will be reduced when your data is stored in a few larger StoreFiles, to avoid extra disk IO during low-level scans to find a specific row.
Bloom filters need to be rebuilt upon deletion, so may not be appropriate in environments with a large number of deletions.
@ -345,7 +342,7 @@ Bloom filters need to be rebuilt upon deletion, so may not be appropriate in env
Bloom filters are enabled on a Column Family.
You can do this by using the setBloomFilterType method of HColumnDescriptor or using the HBase API.
Valid values are `NONE` (the default), `ROW`, or `ROWCOL`.
See <<bloom.filters.when,bloom.filters.when>> for more information on `ROW` versus `ROWCOL`.
See <<bloom.filters.when>> for more information on `ROW` versus `ROWCOL`.
See also the API documentation for link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor].
The following example creates a table and enables a ROWCOL Bloom filter on the `colfam1` column family.
@ -357,7 +354,7 @@ hbase> create 'mytable',{NAME => 'colfam1', BLOOMFILTER => 'ROWCOL'}
==== Configuring Server-Wide Behavior of Bloom Filters
You can configure the following settings in the _hbase-site.xml_.
You can configure the following settings in the _hbase-site.xml_.
[cols="1,1,1", options="header"]
|===
@ -367,8 +364,7 @@ You can configure the following settings in the _hbase-site.xml_.
| io.hfile.bloom.enabled
| yes
| Set to no to kill bloom filters server-wide if
something goes wrong
| Set to no to kill bloom filters server-wide if something goes wrong
| io.hfile.bloom.error.rate
| .01
@ -383,18 +379,16 @@ You can configure the following settings in the _hbase-site.xml_.
| io.storefile.bloom.max.keys
| 128000000
| For default (single-block) Bloom filters, this specifies the maximum
number of keys.
| For default (single-block) Bloom filters, this specifies the maximum number of keys.
| io.storefile.delete.family.bloom.enabled
| true
| Master switch to enable Delete Family Bloom filters and store them in
the StoreFile.
| Master switch to enable Delete Family Bloom filters and store them in the StoreFile.
| io.storefile.bloom.block.size
| 65536
| Target Bloom block size. Bloom filter blocks of approximately this size
are interleaved with data blocks.
are interleaved with data blocks.
| hfile.block.bloom.cacheonwrite
| false
@ -404,35 +398,35 @@ You can configure the following settings in the _hbase-site.xml_.
[[schema.cf.blocksize]]
=== ColumnFamily BlockSize
The blocksize can be configured for each ColumnFamily in a table, and this defaults to 64k.
The blocksize can be configured for each ColumnFamily in a table, and defaults to 64k.
Larger cell values require larger blocksizes.
There is an inverse relationship between blocksize and the resulting StoreFile indexes (i.e., if the blocksize is doubled then the resulting indexes should be roughly halved).
There is an inverse relationship between blocksize and the resulting StoreFile indexes (i.e., if the blocksize is doubled then the resulting indexes should be roughly halved).
See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor] and <<store,store>>for more information.
See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor] and <<store>>for more information.
[[cf.in.memory]]
=== In-Memory ColumnFamilies
ColumnFamilies can optionally be defined as in-memory.
Data is still persisted to disk, just like any other ColumnFamily.
In-memory blocks have the highest priority in the <<block.cache,block.cache>>, but it is not a guarantee that the entire table will be in memory.
In-memory blocks have the highest priority in the <<block.cache>>, but it is not a guarantee that the entire table will be in memory.
See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor] for more information.
See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor] for more information.
[[perf.compression]]
=== Compression
Production systems should use compression with their ColumnFamily definitions.
See <<compression,compression>> for more information.
See <<compression>> for more information.
[[perf.compression.however]]
==== However...
Compression deflates data _on disk_.
When it's in-memory (e.g., in the MemStore) or on the wire (e.g., transferring between RegionServer and Client) it's inflated.
So while using ColumnFamily compression is a best practice, but it's not going to completely eliminate the impact of over-sized Keys, over-sized ColumnFamily names, or over-sized Column names.
So while using ColumnFamily compression is a best practice, but it's not going to completely eliminate the impact of over-sized Keys, over-sized ColumnFamily names, or over-sized Column names.
See <<keysize,keysize>> on for schema design tips, and <<keyvalue,keyvalue>> for more information on HBase stores data internally.
See <<keysize>> on for schema design tips, and <<keyvalue>> for more information on HBase stores data internally.
[[perf.general]]
== HBase General Patterns
@ -444,7 +438,6 @@ When people get started with HBase they have a tendency to write code that looks
[source,java]
----
Get get = new Get(rowkey);
Result r = htable.get(get);
byte[] b = r.getValue(Bytes.toBytes("cf"), Bytes.toBytes("attr")); // returns current version of value
@ -455,7 +448,6 @@ It's better to use constants for the byte-arrays, like this:
[source,java]
----
public static final byte[] CF = "cf".getBytes();
public static final byte[] ATTR = "attr".getBytes();
...
@ -471,61 +463,60 @@ byte[] b = r.getValue(CF, ATTR); // returns current version of value
=== Batch Loading
Use the bulk load tool if you can.
See <<arch.bulk.load,arch.bulk.load>>.
Otherwise, pay attention to the below.
See <<arch.bulk.load>>.
Otherwise, pay attention to the below.
[[precreate.regions]]
=== Table Creation: Pre-Creating Regions
=== Table Creation: Pre-Creating Regions
Tables in HBase are initially created with one region by default.
For bulk imports, this means that all clients will write to the same region until it is large enough to split and become distributed across the cluster.
A useful pattern to speed up the bulk import process is to pre-create empty regions.
Be somewhat conservative in this, because too-many regions can actually degrade performance.
Be somewhat conservative in this, because too-many regions can actually degrade performance.
There are two different approaches to pre-creating splits.
The first approach is to rely on the default `HBaseAdmin` strategy (which is implemented in `Bytes.split`)...
The first approach is to rely on the default `HBaseAdmin` strategy (which is implemented in `Bytes.split`)...
[source,java]
----
byte[] startKey = ...; // your lowest key
byte[] endKey = ...; // your highest key
int numberOfRegions = ...; // # of regions to create
byte[] startKey = ...; // your lowest key
byte[] endKey = ...; // your highest key
int numberOfRegions = ...; // # of regions to create
admin.createTable(table, startKey, endKey, numberOfRegions);
----
And the other approach is to define the splits yourself...
And the other approach is to define the splits yourself...
[source,java]
----
byte[][] splits = ...; // create your own splits
admin.createTable(table, splits);
----
See <<rowkey.regionsplits,rowkey.regionsplits>> for issues related to understanding your keyspace and pre-creating regions.
See <<manual_region_splitting_decisions,manual region splitting decisions>> for discussion on manually pre-splitting regions.
See <<rowkey.regionsplits>> for issues related to understanding your keyspace and pre-creating regions.
See <<manual_region_splitting_decisions,manual region splitting decisions>> for discussion on manually pre-splitting regions.
[[def.log.flush]]
=== Table Creation: Deferred Log Flush
=== Table Creation: Deferred Log Flush
The default behavior for Puts using the Write Ahead Log (WAL) is that `WAL` edits will be written immediately.
If deferred log flush is used, WAL edits are kept in memory until the flush period.
The benefit is aggregated and asynchronous `WAL`- writes, but the potential downside is that if the RegionServer goes down the yet-to-be-flushed edits are lost.
This is safer, however, than not using WAL at all with Puts.
This is safer, however, than not using WAL at all with Puts.
Deferred log flush can be configured on tables via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HTableDescriptor.html[HTableDescriptor].
The default value of `hbase.regionserver.optionallogflushinterval` is 1000ms.
The default value of `hbase.regionserver.optionallogflushinterval` is 1000ms.
[[perf.hbase.client.autoflush]]
=== HBase Client: AutoFlush
When performing a lot of Puts, make sure that setAutoFlush is set to false on your link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html[HTable] instance.
When performing a lot of Puts, make sure that setAutoFlush is set to false on your link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html[HTable] instance.
Otherwise, the Puts will be sent one at a time to the RegionServer.
Puts added via ` htable.add(Put)` and ` htable.add( <List> Put)` wind up in the same write buffer.
Puts added via `htable.add(Put)` and `htable.add( <List> Put)` wind up in the same write buffer.
If `autoFlush = false`, these messages are not sent until the write-buffer is filled.
To explicitly flush the messages, call [method]+flushCommits+.
Calling [method]+close+ on the `HTable` instance will invoke [method]+flushCommits+.
To explicitly flush the messages, call `flushCommits`.
Calling `close` on the `HTable` instance will invoke `flushCommits`.
[[perf.hbase.client.putwal]]
=== HBase Client: Turn off WAL on Puts
@ -536,47 +527,46 @@ Bulk loads can be re-run in the event of a crash, with little risk of data loss.
WARNING: If you disable the WAL for anything other than bulk loads, your data is at risk.
In general, it is best to use WAL for Puts, and where loading throughput is a concern to use link:[bulk loading] techniques instead.
In general, it is best to use WAL for Puts, and where loading throughput is a concern to use bulk loading techniques instead.
For normal Puts, you are not likely to see a performance improvement which would outweigh the risk.
To disable the WAL, see <<wal.disable,wal.disable>>.
To disable the WAL, see <<wal.disable>>.
[[perf.hbase.client.regiongroup]]
=== HBase Client: Group Puts by RegionServer
In addition to using the writeBuffer, grouping `Put`s by RegionServer can reduce the number of client RPC calls per writeBuffer flush.
There is a utility `HTableUtil` currently on TRUNK that does this, but you can either copy that or implement your own version for those still on 0.90.x or earlier.
There is a utility `HTableUtil` currently on TRUNK that does this, but you can either copy that or implement your own version for those still on 0.90.x or earlier.
[[perf.hbase.write.mr.reducer]]
=== MapReduce: Skip The Reducer
When writing a lot of data to an HBase table from a MR job (e.g., with link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.html[TableOutputFormat]), and specifically where Puts are being emitted from the Mapper, skip the Reducer step.
When a Reducer step is used, all of the output (Puts) from the Mapper will get spooled to disk, then sorted/shuffled to other Reducers that will most likely be off-node.
It's far more efficient to just write directly to HBase.
It's far more efficient to just write directly to HBase.
For summary jobs where HBase is used as a source and a sink, then writes will be coming from the Reducer step (e.g., summarize values then write out result). This is a different processing problem than from the the above case.
For summary jobs where HBase is used as a source and a sink, then writes will be coming from the Reducer step (e.g., summarize values then write out result). This is a different processing problem than from the the above case.
[[perf.one.region]]
=== Anti-Pattern: One Hot Region
If all your data is being written to one region at a time, then re-read the section on processing link:[timeseries] data.
If all your data is being written to one region at a time, then re-read the section on processing timeseries data.
Also, if you are pre-splitting regions and all your data is _still_ winding up in a single region even though your keys aren't monotonically increasing, confirm that your keyspace actually works with the split strategy.
Also, if you are pre-splitting regions and all your data is _still_ winding up in a single region even though your keys aren't monotonically increasing, confirm that your keyspace actually works with the split strategy.
There are a variety of reasons that regions may appear "well split" but won't work with your data.
As the HBase client communicates directly with the RegionServers, this can be obtained via link:hhttp://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#getRegionLocation(byte[])[HTable.getRegionLocation].
As the HBase client communicates directly with the RegionServers, this can be obtained via link:hhttp://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#getRegionLocation(byte[])[HTable.getRegionLocation].
See <<precreate.regions,precreate.regions>>, as well as <<perf.configurations,perf.configurations>>
See <<precreate.regions>>, as well as <<perf.configurations>>
[[perf.reading]]
== Reading from HBase
The mailing list can help if you are having performance issues.
For example, here is a good general thread on what to look at addressing read-time issues: link:http://search-hadoop.com/m/qOo2yyHtCC1[HBase Random Read latency >
100ms]
For example, here is a good general thread on what to look at addressing read-time issues: link:http://search-hadoop.com/m/qOo2yyHtCC1[HBase Random Read latency > 100ms]
[[perf.hbase.client.caching]]
=== Scan Caching
If HBase is used as an input source for a MapReduce job, for example, make sure that the input link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scan] instance to the MapReduce job has [method]+setCaching+ set to something greater than the default (which is 1). Using the default value means that the map-task will make call back to the region-server for every record processed.
If HBase is used as an input source for a MapReduce job, for example, make sure that the input link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scan] instance to the MapReduce job has `setCaching` set to something greater than the default (which is 1). Using the default value means that the map-task will make call back to the region-server for every record processed.
Setting this value to 500, for example, will transfer 500 rows at a time to the client to be processed.
There is a cost/benefit to have the cache value be large because it costs more in memory for both client and RegionServer, so bigger isn't always better.
@ -585,18 +575,18 @@ There is a cost/benefit to have the cache value be large because it costs more i
Scan settings in MapReduce jobs deserve special attention.
Timeouts can result (e.g., UnknownScannerException) in Map tasks if it takes longer to process a batch of records before the client goes back to the RegionServer for the next set of data.
This problem can occur because there is non-trivial processing occuring per row.
This problem can occur because there is non-trivial processing occurring per row.
If you process rows quickly, set caching higher.
If you process rows more slowly (e.g., lots of transformations per row, writes), then set caching lower.
If you process rows more slowly (e.g., lots of transformations per row, writes), then set caching lower.
Timeouts can also happen in a non-MapReduce use case (i.e., single threaded HBase client doing a Scan), but the processing that is often performed in MapReduce jobs tends to exacerbate this issue.
Timeouts can also happen in a non-MapReduce use case (i.e., single threaded HBase client doing a Scan), but the processing that is often performed in MapReduce jobs tends to exacerbate this issue.
[[perf.hbase.client.selection]]
=== Scan Attribute Selection
Whenever a Scan is used to process large numbers of rows (and especially when used as a MapReduce source), be aware of which attributes are selected.
If `scan.addFamily` is called then _all_ of the attributes in the specified ColumnFamily will be returned to the client.
If only a small number of the available attributes are to be processed, then only those attributes should be specified in the input scan because attribute over-selection is a non-trivial performance penalty over large datasets.
If `scan.addFamily` is called then _all_ of the attributes in the specified ColumnFamily will be returned to the client.
If only a small number of the available attributes are to be processed, then only those attributes should be specified in the input scan because attribute over-selection is a non-trivial performance penalty over large datasets.
[[perf.hbase.client.seek]]
=== Avoid scan seeks
@ -610,7 +600,6 @@ The following code instructs the RegionServer to attempt two iterations of next
[source,java]
----
Scan scan = new Scan();
scan.addColumn(...);
scan.setAttribute(Scan.HINT_LOOKAHEAD, Bytes.toBytes(2));
@ -620,18 +609,17 @@ table.getScanner(scan);
[[perf.hbase.mr.input]]
=== MapReduce - Input Splits
For MapReduce jobs that use HBase tables as a source, if there a pattern where the "slow" map tasks seem to have the same Input Split (i.e., the RegionServer serving the data), see the Troubleshooting Case Study in <<casestudies.slownode,casestudies.slownode>>.
For MapReduce jobs that use HBase tables as a source, if there a pattern where the "slow" map tasks seem to have the same Input Split (i.e., the RegionServer serving the data), see the Troubleshooting Case Study in <<casestudies.slownode>>.
[[perf.hbase.client.scannerclose]]
=== Close ResultScanners
This isn't so much about improving performance but rather _avoiding_ performance problems.
If you forget to close link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/ResultScanner.html[ResultScanners] you can cause problems on the RegionServers.
Always have ResultScanner processing enclosed in try/catch blocks...
This isn't so much about improving performance but rather _avoiding_ performance problems.
If you forget to close link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/ResultScanner.html[ResultScanners] you can cause problems on the RegionServers.
Always have ResultScanner processing enclosed in try/catch blocks.
[source,java]
----
Scan scan = new Scan();
// set attrs...
ResultScanner rs = htable.getScanner(scan);
@ -647,44 +635,42 @@ htable.close();
[[perf.hbase.client.blockcache]]
=== Block Cache
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scan] instances can be set to use the block cache in the RegionServer via the [method]+setCacheBlocks+ method.
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scan] instances can be set to use the block cache in the RegionServer via the `setCacheBlocks` method.
For input Scans to MapReduce jobs, this should be `false`.
For frequently accessed rows, it is advisable to use the block cache.
Cache more data by moving your Block Cache offheap.
See <<offheap.blockcache,offheap.blockcache>>
Cache more data by moving your Block Cache off-heap.
See <<offheap.blockcache>>
[[perf.hbase.client.rowkeyonly]]
=== Optimal Loading of Row Keys
When performing a table link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[scan] where only the row keys are needed (no families, qualifiers, values or timestamps), add a FilterList with a `MUST_PASS_ALL` operator to the scanner using [method]+setFilter+.
The filter list should include both a link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.html[FirstKeyOnlyFilter] and a link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/KeyOnlyFilter.html[KeyOnlyFilter].
Using this filter combination will result in a worst case scenario of a RegionServer reading a single value from disk and minimal network traffic to the client for a single row.
When performing a table link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[scan] where only the row keys are needed (no families, qualifiers, values or timestamps), add a FilterList with a `MUST_PASS_ALL` operator to the scanner using `setFilter`.
The filter list should include both a link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.html[FirstKeyOnlyFilter] and a link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/KeyOnlyFilter.html[KeyOnlyFilter].
Using this filter combination will result in a worst case scenario of a RegionServer reading a single value from disk and minimal network traffic to the client for a single row.
[[perf.hbase.read.dist]]
=== Concurrency: Monitor Data Spread
When performing a high number of concurrent reads, monitor the data spread of the target tables.
If the target table(s) have too few regions then the reads could likely be served from too few nodes.
If the target table(s) have too few regions then the reads could likely be served from too few nodes.
See <<precreate.regions,precreate.regions>>, as well as <<perf.configurations,perf.configurations>>
See <<precreate.regions>>, as well as <<perf.configurations>>
[[blooms]]
=== Bloom Filters
Enabling Bloom Filters can save your having to go to disk and can help improve read latencies.
link:http://en.wikipedia.org/wiki/Bloom_filter[Bloom filters] were developed over in link:https://issues.apache.org/jira/browse/HBASE-1200[HBase-1200 Add
bloomfilters].
For description of the development process -- why static blooms rather than dynamic -- and for an overview of the unique properties that pertain to blooms in HBase, as well as possible future directions, see the _Development Process_ section of the document link:https://issues.apache.org/jira/secure/attachment/12444007/Bloom_Filters_in_HBase.pdf[BloomFilters
in HBase] attached to link:https://issues.apache.org/jira/browse/HBASE-1200[HBase-1200].
link:http://en.wikipedia.org/wiki/Bloom_filter[Bloom filters] were developed over in link:https://issues.apache.org/jira/browse/HBASE-1200[HBase-1200 Add bloomfilters].
For description of the development process -- why static blooms rather than dynamic -- and for an overview of the unique properties that pertain to blooms in HBase, as well as possible future directions, see the _Development Process_ section of the document link:https://issues.apache.org/jira/secure/attachment/12444007/Bloom_Filters_in_HBase.pdf[BloomFilters in HBase] attached to link:https://issues.apache.org/jira/browse/HBASE-1200[HBASE-1200].
The bloom filters described here are actually version two of blooms in HBase.
In versions up to 0.19.x, HBase had a dynamic bloom option based on work done by the link:http://www.one-lab.org[European Commission One-Lab Project 034819].
The core of the HBase bloom work was later pulled up into Hadoop to implement org.apache.hadoop.io.BloomMapFile.
Version 1 of HBase blooms never worked that well.
Version 2 is a rewrite from scratch though again it starts with the one-lab work.
See also <<schema.bloom,schema.bloom>>.
See also <<schema.bloom>>.
[[bloom_footprint]]
==== Bloom StoreFile footprint
@ -698,11 +684,11 @@ Bloom filters add an entry to the `StoreFile` general `FileInfo` data structure
===== BloomFilter entries in `StoreFile` metadata
`BLOOM_FILTER_META` holds Bloom Size, Hash Function used, etc.
Its small in size and is cached on `StoreFile.Reader` load
It's small in size and is cached on `StoreFile.Reader` load
`BLOOM_FILTER_DATA` is the actual bloomfilter data.
Obtained on-demand.
Stored in the LRU cache, if it is enabled (Its enabled by default).
Stored in the LRU cache, if it is enabled (It's enabled by default).
[[config.bloom]]
==== Bloom Filter Configuration
@ -723,8 +709,7 @@ to .5%) == +1 bit per bloom entry.
`io.hfile.bloom.max.fold` = guaranteed minimum fold rate.
Most people should leave this alone.
Default = 7, or can collapse to at least 1/128th of original size.
See the _Development Process_ section of the document link:https://issues.apache.org/jira/secure/attachment/12444007/Bloom_Filters_in_HBase.pdf[BloomFilters
in HBase] for more on what this option means.
See the _Development Process_ section of the document link:https://issues.apache.org/jira/secure/attachment/12444007/Bloom_Filters_in_HBase.pdf[BloomFilters in HBase] for more on what this option means.
=== Hedged Reads
@ -736,12 +721,14 @@ Hedged reads can be helpful for times where a rare slow read is caused by a tran
Because a HBase RegionServer is a HDFS client, you can enable hedged reads in HBase, by adding the following properties to the RegionServer's hbase-site.xml and tuning the values to suit your environment.
* .Configuration for Hedged Reads`dfs.client.hedged.read.threadpool.size` - the number of threads dedicated to servicing hedged reads.
.Configuration for Hedged Reads
* `dfs.client.hedged.read.threadpool.size` - the number of threads dedicated to servicing hedged reads.
If this is set to 0 (the default), hedged reads are disabled.
* `dfs.client.hedged.read.threshold.millis` - the number of milliseconds to wait before spawning a second read thread.
.Hedged Reads Configuration Example
====
[source,xml]
----
<property>
<name>dfs.client.hedged.read.threadpool.size</name>
@ -755,9 +742,10 @@ Because a HBase RegionServer is a HDFS client, you can enable hedged reads in HB
====
Use the following metrics to tune the settings for hedged reads on your cluster.
See <<hbase_metrics,hbase metrics>> for more information.
See <<hbase_metrics>> for more information.
* .Metrics for Hedged ReadshedgedReadOps - the number of times hedged read threads have been triggered.
.Metrics for Hedged Reads
* hedgedReadOps - the number of times hedged read threads have been triggered.
This could indicate that read requests are often slow, or that hedged reads are triggered too quickly.
* hedgeReadOpsWin - the number of times the hedged read thread was faster than the original thread.
This could indicate that a given RegionServer is having trouble servicing requests.
@ -770,24 +758,24 @@ See <<hbase_metrics,hbase metrics>> for more information.
HBase tables are sometimes used as queues.
In this case, special care must be taken to regularly perform major compactions on tables used in this manner.
As is documented in <<datamodel,datamodel>>, marking rows as deleted creates additional StoreFiles which then need to be processed on reads.
Tombstones only get cleaned up with major compactions.
As is documented in <<datamodel>>, marking rows as deleted creates additional StoreFiles which then need to be processed on reads.
Tombstones only get cleaned up with major compactions.
See also <<compaction,compaction>> and link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html#majorCompact%28java.lang.String%29[HBaseAdmin.majorCompact].
See also <<compaction>> and link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html#majorCompact%28java.lang.String%29[HBaseAdmin.majorCompact].
[[perf.deleting.rpc]]
=== Delete RPC Behavior
Be aware that `htable.delete(Delete)` doesn't use the writeBuffer.
It will execute an RegionServer RPC with each invocation.
For a large number of deletes, consider `htable.delete(List)`.
For a large number of deletes, consider `htable.delete(List)`.
See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#delete%28org.apache.hadoop.hbase.client.Delete%29
See http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#delete%28org.apache.hadoop.hbase.client.Delete%29
[[perf.hdfs]]
== HDFS
Because HBase runs on <<arch.hdfs,arch.hdfs>> it is important to understand how it works and how it affects HBase.
Because HBase runs on <<arch.hdfs>> it is important to understand how it works and how it affects HBase.
[[perf.hdfs.curr]]
=== Current Issues With Low-Latency Reads
@ -795,26 +783,22 @@ Because HBase runs on <<arch.hdfs,arch.hdfs>> it is important to understand how
The original use-case for HDFS was batch processing.
As such, there low-latency reads were historically not a priority.
With the increased adoption of Apache HBase this is changing, and several improvements are already in development.
See the link:https://issues.apache.org/jira/browse/HDFS-1599[Umbrella Jira Ticket for HDFS
Improvements for HBase].
See the link:https://issues.apache.org/jira/browse/HDFS-1599[Umbrella Jira Ticket for HDFS Improvements for HBase].
[[perf.hdfs.configs.localread]]
=== Leveraging local data
Since Hadoop 1.0.0 (also 0.22.1, 0.23.1, CDH3u3 and HDP 1.0) via link:https://issues.apache.org/jira/browse/HDFS-2246[HDFS-2246], it is possible for the DFSClient to take a "short circuit" and read directly from the disk instead of going through the DataNode when the data is local.
What this means for HBase is that the RegionServers can read directly off their machine's disks instead of having to open a socket to talk to the DataNode, the former being generally much faster.
See JD's link:http://files.meetup.com/1350427/hug_ebay_jdcryans.pdf[Performance
Talk].
Also see link:http://search-hadoop.com/m/zV6dKrLCVh1[HBase, mail # dev - read short
circuit] thread for more discussion around short circuit reads.
See JD's link:http://files.meetup.com/1350427/hug_ebay_jdcryans.pdf[Performance Talk].
Also see link:http://search-hadoop.com/m/zV6dKrLCVh1[HBase, mail # dev - read short circuit] thread for more discussion around short circuit reads.
To enable "short circuit" reads, it will depend on your version of Hadoop.
The original shortcircuit read patch was much improved upon in Hadoop 2 in link:https://issues.apache.org/jira/browse/HDFS-347[HDFS-347].
See link:http://blog.cloudera.com/blog/2013/08/how-improved-short-circuit-local-reads-bring-better-performance-and-security-to-hadoop/ for details on the difference between the old and new implementations.
See link:http://archive.cloudera.com/cdh4/cdh/4/hadoop/hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html[Hadoop
shortcircuit reads configuration page] for how to enable the latter, better version of shortcircuit.
See http://blog.cloudera.com/blog/2013/08/how-improved-short-circuit-local-reads-bring-better-performance-and-security-to-hadoop/ for details on the difference between the old and new implementations.
See link:http://archive.cloudera.com/cdh4/cdh/4/hadoop/hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html[Hadoop shortcircuit reads configuration page] for how to enable the latter, better version of shortcircuit.
For example, here is a minimal config.
enabling short-circuit reads added to _hbase-site.xml_:
enabling short-circuit reads added to _hbase-site.xml_:
[source,xml]
----
@ -837,38 +821,37 @@ enabling short-circuit reads added to _hbase-site.xml_:
</property>
----
Be careful about permissions for the directory that hosts the shared domain socket; dfsclient will complain if open to other than the hbase user.
Be careful about permissions for the directory that hosts the shared domain socket; dfsclient will complain if open to other than the hbase user.
If you are running on an old Hadoop, one that is without link:https://issues.apache.org/jira/browse/HDFS-347[HDFS-347] but that has link:https://issues.apache.org/jira/browse/HDFS-2246[HDFS-2246], you must set two configurations.
First, the hdfs-site.xml needs to be amended.
Set the property `dfs.block.local-path-access.user` to be the _only_ user that can use the shortcut.
Set the property `dfs.block.local-path-access.user` to be the _only_ user that can use the shortcut.
This has to be the user that started HBase.
Then in hbase-site.xml, set `dfs.client.read.shortcircuit` to be `true`
Then in hbase-site.xml, set `dfs.client.read.shortcircuit` to be `true`
Services -- at least the HBase RegionServers -- will need to be restarted in order to pick up the new configurations.
Services -- at least the HBase RegionServers -- will need to be restarted in order to pick up the new configurations.
.dfs.client.read.shortcircuit.buffer.size
[NOTE]
====
The default for this value is too high when running on a highly trafficed HBase.
In HBase, if this value has not been set, we set it down from the default of 1M to 128k (Since HBase 0.98.0 and 0.96.1). See link:https://issues.apache.org/jira/browse/HBASE-8143[HBASE-8143 HBase on Hadoop
2 with local short circuit reads (ssr) causes OOM]). The Hadoop DFSClient in HBase will allocate a direct byte buffer of this size for _each_ block it has open; given HBase keeps its HDFS files open all the time, this can add up quickly.
The default for this value is too high when running on a highly trafficked HBase.
In HBase, if this value has not been set, we set it down from the default of 1M to 128k (Since HBase 0.98.0 and 0.96.1). See link:https://issues.apache.org/jira/browse/HBASE-8143[HBASE-8143 HBase on Hadoop 2 with local short circuit reads (ssr) causes OOM]). The Hadoop DFSClient in HBase will allocate a direct byte buffer of this size for _each_ block it has open; given HBase keeps its HDFS files open all the time, this can add up quickly.
====
[[perf.hdfs.comp]]
=== Performance Comparisons of HBase vs. HDFS
A fairly common question on the dist-list is why HBase isn't as performant as HDFS files in a batch context (e.g., as a MapReduce source or sink). The short answer is that HBase is doing a lot more than HDFS (e.g., reading the KeyValues, returning the most current row or specified timestamps, etc.), and as such HBase is 4-5 times slower than HDFS in this processing context.
There is room for improvement and this gap will, over time, be reduced, but HDFS will always be faster in this use-case.
There is room for improvement and this gap will, over time, be reduced, but HDFS will always be faster in this use-case.
[[perf.ec2]]
== Amazon EC2
Performance questions are common on Amazon EC2 environments because it is a shared environment.
You will not see the same throughput as a dedicated server.
In terms of running tests on EC2, run them several times for the same reason (i.e., it's a shared environment and you don't know what else is happening on the server).
In terms of running tests on EC2, run them several times for the same reason (i.e., it's a shared environment and you don't know what else is happening on the server).
If you are running on EC2 and post performance questions on the dist-list, please state this fact up-front that because EC2 issues are practically a separate class of performance issues.
If you are running on EC2 and post performance questions on the dist-list, please state this fact up-front that because EC2 issues are practically a separate class of performance issues.
[[perf.hbase.mr.cluster]]
== Collocating HBase and MapReduce
@ -877,17 +860,17 @@ It is often recommended to have different clusters for HBase and MapReduce.
A better qualification of this is: don't collocate a HBase that serves live requests with a heavy MR workload.
OLTP and OLAP-optimized systems have conflicting requirements and one will lose to the other, usually the former.
For example, short latency-sensitive disk reads will have to wait in line behind longer reads that are trying to squeeze out as much throughput as possible.
MR jobs that write to HBase will also generate flushes and compactions, which will in turn invalidate blocks in the <<block.cache,block.cache>>.
MR jobs that write to HBase will also generate flushes and compactions, which will in turn invalidate blocks in the <<block.cache>>.
If you need to process the data from your live HBase cluster in MR, you can ship the deltas with <<copy.table,copy.table>> or use replication to get the new data in real time on the OLAP cluster.
In the worst case, if you really need to collocate both, set MR to use less Map and Reduce slots than you'd normally configure, possibly just one.
If you need to process the data from your live HBase cluster in MR, you can ship the deltas with <<copy.table>> or use replication to get the new data in real time on the OLAP cluster.
In the worst case, if you really need to collocate both, set MR to use less Map and Reduce slots than you'd normally configure, possibly just one.
When HBase is used for OLAP operations, it's preferable to set it up in a hardened way like configuring the ZooKeeper session timeout higher and giving more memory to the MemStores (the argument being that the Block Cache won't be used much since the workloads are usually long scans).
When HBase is used for OLAP operations, it's preferable to set it up in a hardened way like configuring the ZooKeeper session timeout higher and giving more memory to the MemStores (the argument being that the Block Cache won't be used much since the workloads are usually long scans).
[[perf.casestudy]]
== Case Studies
For Performance and Troubleshooting Case Studies, see <<casestudies,casestudies>>.
For Performance and Troubleshooting Case Studies, see <<casestudies>>.
ifdef::backend-docbook[]
[index]

View File

@ -29,25 +29,20 @@
This is the official reference guide for the link:http://hbase.apache.org/[HBase] version it ships with.
Herein you will find either the definitive documentation on an HBase topic as of its standing when the referenced HBase version shipped, or it will point to the location in link:http://hbase.apache.org/apidocs/index.html[javadoc], link:https://issues.apache.org/jira/browse/HBASE[JIRA] or link:http://wiki.apache.org/hadoop/Hbase[wiki] where the pertinent information can be found.
Herein you will find either the definitive documentation on an HBase topic as of its standing when the referenced HBase version shipped, or it will point to the location in link:http://hbase.apache.org/apidocs/index.html[Javadoc], link:https://issues.apache.org/jira/browse/HBASE[JIRA] or link:http://wiki.apache.org/hadoop/Hbase[wiki] where the pertinent information can be found.
.About This Guide
This reference guide is a work in progress. The source for this guide can be found in the _src/main/dasciidoc_ directory of the HBase source. This reference guide is marked up using Asciidoc, from which the the finished guide is generated as part of the 'site' build target. Run
This reference guide is a work in progress. The source for this guide can be found in the _src/main/asciidoc directory of the HBase source. This reference guide is marked up using link:http://asciidoc.org/[AsciiDoc] from which the finished guide is generated as part of the 'site' build target. Run
[source,bourne]
----
mvn site
----
----
to generate this documentation.
Amendments and improvements to the documentation are welcomed.
Click link:https://issues.apache.org/jira/secure/CreateIssueDetails!init.jspa?pid=12310753&issuetype=1&components=12312132&summary=SHORT+DESCRIPTION[this link] to file a new documentation bug against Apache HBase with some values pre-selected.
.Contributing to the Documentation
For an overview of Asciidoc and suggestions to get started contributing to the documentation, see <<appendix_contributing_to_documentation,appendix contributing to documentation>>.
.Providing Feedback
This guide allows you to leave comments or questions on any page, using Disqus.
Look for the Comments area at the bottom of the page.
Answering these questions is a volunteer effort, and may be delayed.
For an overview of AsciiDoc and suggestions to get started contributing to the documentation, see the <<appendix_contributing_to_documentation,relevant section later in this documentation>>.
.Heads-up if this is your first foray into the world of distributed computing...
If this is your first foray into the wonderful world of Distributed Computing, then you are in for some interesting times.
@ -57,8 +52,8 @@ Your cluster's operation can hiccup because of any of a myriad set of reasons fr
Here is one good starting point: link:http://en.wikipedia.org/wiki/Fallacies_of_Distributed_Computing[Fallacies of Distributed Computing].
That said, you are welcome. +
Its a fun place to be. +
Yours, the HBase Community.
It's a fun place to be. +
Yours, the HBase Community.
:numbered:

View File

@ -27,15 +27,12 @@
:icons: font
:experimental:
A good general introduction on the strength and weaknesses modelling on the various non-rdbms datastores is Ian Varley's Master thesis, link:http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf[No Relation:
The Mixed Blessings of Non-Relational Databases].
Recommended.
Also, read <<keyvalue,keyvalue>> for how HBase stores data internally, and the section on <<schema.casestudies,schema.casestudies>>.
A good general introduction on the strength and weaknesses modelling on the various non-rdbms datastores is Ian Varley's Master thesis, link:http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf[No Relation: The Mixed Blessings of Non-Relational Databases]. Also, read <<keyvalue,keyvalue>> for how HBase stores data internally, and the section on <<schema.casestudies,schema.casestudies>>.
[[schema.creation]]
== Schema Creation
== Schema Creation
HBase schemas can be created or updated with <<shell,shell>> or by using link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html[HBaseAdmin] in the Java API.
HBase schemas can be created or updated using the <<shell>> or by using link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html[HBaseAdmin] in the Java API.
Tables must be disabled when making ColumnFamily modifications, for example:
@ -58,30 +55,30 @@ admin.enableTable(table);
See <<client_dependencies,client dependencies>> for more information about configuring client connections.
Note: online schema changes are supported in the 0.92.x codebase, but the 0.90.x codebase requires the table to be disabled.
NOTE: online schema changes are supported in the 0.92.x codebase, but the 0.90.x codebase requires the table to be disabled.
[[schema.updates]]
=== Schema Updates
When changes are made to either Tables or ColumnFamilies (e.g., region size, block size), these changes take effect the next time there is a major compaction and the StoreFiles get re-written.
When changes are made to either Tables or ColumnFamilies (e.g. region size, block size), these changes take effect the next time there is a major compaction and the StoreFiles get re-written.
See <<store,store>> for more information on StoreFiles.
See <<store,store>> for more information on StoreFiles.
[[number.of.cfs]]
== On the number of column families
== On the number of column families
HBase currently does not do well with anything above two or three column families so keep the number of column families in your schema low.
Currently, flushing and compactions are done on a per Region basis so if one column family is carrying the bulk of the data bringing on flushes, the adjacent families will also be flushed though the amount of data they carry is small.
When many column families the flushing and compaction interaction can make for a bunch of needless i/o loading (To be addressed by changing flushing and compaction to work on a per column family basis). For more information on compactions, see <<compaction,compaction>>.
Currently, flushing and compactions are done on a per Region basis so if one column family is carrying the bulk of the data bringing on flushes, the adjacent families will also be flushed even though the amount of data they carry is small.
When many column families exist the flushing and compaction interaction can make for a bunch of needless i/o (To be addressed by changing flushing and compaction to work on a per column family basis). For more information on compactions, see <<compaction>>.
Try to make do with one column family if you can in your schemas.
Only introduce a second and third column family in the case where data access is usually column scoped; i.e.
you query one column family or the other but usually not both at the one time.
you query one column family or the other but usually not both at the one time.
[[number.of.cfs.card]]
=== Cardinality of ColumnFamilies
Where multiple ColumnFamilies exist in a single table, be aware of the cardinality (i.e., number of rows). If ColumnFamilyA has 1 million rows and ColumnFamilyB has 1 billion rows, ColumnFamilyA's data will likely be spread across many, many regions (and RegionServers). This makes mass scans for ColumnFamilyA less efficient.
Where multiple ColumnFamilies exist in a single table, be aware of the cardinality (i.e., number of rows). If ColumnFamilyA has 1 million rows and ColumnFamilyB has 1 billion rows, ColumnFamilyA's data will likely be spread across many, many regions (and RegionServers). This makes mass scans for ColumnFamilyA less efficient.
[[rowkey.design]]
== Rowkey Design
@ -105,7 +102,7 @@ Salting in this sense has nothing to do with cryptography, but refers to adding
In this case, salting refers to adding a randomly-assigned prefix to the row key to cause it to sort differently than it otherwise would.
The number of possible prefixes correspond to the number of regions you want to spread the data across.
Salting can be helpful if you have a few "hot" row key patterns which come up over and over amongst other more evenly-distributed rows.
Consider the following example, which shows that salting can spread write load across multiple regionservers, and illustrates some of the negative implications for reads.
Consider the following example, which shows that salting can spread write load across multiple RegionServers, and illustrates some of the negative implications for reads.
.Salting Example
====
@ -154,7 +151,7 @@ In this way, salting attempts to increase throughput on writes, but has a cost d
.Hashing
Instead of a random assignment, you could use a one-way [firstterm]_hash_ that would cause a given row to always be "salted" with the same prefix, in a way that would spread the load across the regionservers, but allow for predictability during reads.
Instead of a random assignment, you could use a one-way [firstterm]_hash_ that would cause a given row to always be "salted" with the same prefix, in a way that would spread the load across the RegionServers, but allow for predictability during reads.
Using a deterministic hash allows the client to reconstruct the complete rowkey and use a Get operation to retrieve that row as normal.
.Hashing Example
@ -167,71 +164,66 @@ You could also optimize things so that certain pairs of keys were always in the
A third common trick for preventing hotspotting is to reverse a fixed-width or numeric row key so that the part that changes the most often (the least significant digit) is first.
This effectively randomizes row keys, but sacrifices row ordering properties.
See link:https://communities.intel.com/community/itpeernetwork/datastack/blog/2013/11/10/discussion-on-designing-hbase-tables, and link:http://phoenix.apache.org/salted.html[article on Salted Tables] from the Phoenix project, and the discussion in the comments of link:https://issues.apache.org/jira/browse/HBASE-11682[HBASE-11682] for more information about avoiding hotspotting.
See https://communities.intel.com/community/itpeernetwork/datastack/blog/2013/11/10/discussion-on-designing-hbase-tables, and link:http://phoenix.apache.org/salted.html[article on Salted Tables] from the Phoenix project, and the discussion in the comments of link:https://issues.apache.org/jira/browse/HBASE-11682[HBASE-11682] for more information about avoiding hotspotting.
[[timeseries]]
=== Monotonically Increasing Row Keys/Timeseries Data
=== Monotonically Increasing Row Keys/Timeseries Data
In the HBase chapter of Tom White's book link:http://oreilly.com/catalog/9780596521981[Hadoop: The Definitive Guide] (O'Reilly) there is a an optimization note on watching out for a phenomenon where an import process walks in lock-step with all clients in concert pounding one of the table's regions (and thus, a single node), then moving onto the next region, etc.
In the HBase chapter of Tom White's book link:http://oreilly.com/catalog/9780596521981[Hadoop: The Definitive Guide] (O'Reilly) there is a an optimization note on watching out for a phenomenon where an import process walks in lock-step with all clients in concert pounding one of the table's regions (and thus, a single node), then moving onto the next region, etc.
With monotonically increasing row-keys (i.e., using a timestamp), this will happen.
See this comic by IKai Lan on why monotonically increasing row keys are problematic in BigTable-like datastores: link:http://ikaisays.com/2011/01/25/app-engine-datastore-tip-monotonically-increasing-values-are-bad/[monotonically
increasing values are bad].
The pile-up on a single region brought on by monotonically increasing keys can be mitigated by randomizing the input records to not be in sorted order, but in general it's best to avoid using a timestamp or a sequence (e.g.
1, 2, 3) as the row-key.
See this comic by IKai Lan on why monotonically increasing row keys are problematic in BigTable-like datastores: link:http://ikaisays.com/2011/01/25/app-engine-datastore-tip-monotonically-increasing-values-are-bad/[monotonically increasing values are bad].
The pile-up on a single region brought on by monotonically increasing keys can be mitigated by randomizing the input records to not be in sorted order, but in general it's best to avoid using a timestamp or a sequence (e.g. 1, 2, 3) as the row-key.
If you do need to upload time series data into HBase, you should study link:http://opentsdb.net/[OpenTSDB] as a successful example.
It has a page describing the link: http://opentsdb.net/schema.html[schema] it uses in HBase.
The key format in OpenTSDB is effectively [metric_type][event_timestamp], which would appear at first glance to contradict the previous advice about not using a timestamp as the key.
However, the difference is that the timestamp is not in the _lead_ position of the key, and the design assumption is that there are dozens or hundreds (or more) of different metric types.
Thus, even with a continual stream of input data with a mix of metric types, the Puts are distributed across various points of regions in the table.
However, the difference is that the timestamp is not in the _lead_ position of the key, and the design assumption is that there are dozens or hundreds (or more) of different metric types.
Thus, even with a continual stream of input data with a mix of metric types, the Puts are distributed across various points of regions in the table.
See <<schema.casestudies,schema.casestudies>> for some rowkey design examples.
See <<schema.casestudies,schema.casestudies>> for some rowkey design examples.
[[keysize]]
=== Try to minimize row and column sizes
In HBase, values are always freighted with their coordinates; as a cell value passes through the system, it'll be accompanied by its row, column name, and timestamp - always.
If your rows and column names are large, especially compared to the size of the cell value, then you may run up against some interesting scenarios.
One such is the case described by Marc Limotte at the tail of link:https://issues.apache.org/jira/browse/HBASE-3551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13005272#comment-13005272[HBASE-3551] (recommended!). Therein, the indices that are kept on HBase storefiles (<<hfile,hfile>>) to facilitate random access may end up occupyng large chunks of the HBase allotted RAM because the cell value coordinates are large.
One such is the case described by Marc Limotte at the tail of link:https://issues.apache.org/jira/browse/HBASE-3551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13005272#comment-13005272[HBASE-3551] (recommended!). Therein, the indices that are kept on HBase storefiles (<<hfile>>) to facilitate random access may end up occupying large chunks of the HBase allotted RAM because the cell value coordinates are large.
Mark in the above cited comment suggests upping the block size so entries in the store file index happen at a larger interval or modify the table schema so it makes for smaller rows and column names.
Compression will also make for larger indices.
See the thread link:http://search-hadoop.com/m/hemBv1LiN4Q1/a+question+storefileIndexSize&subj=a+question+storefileIndexSize[a
question storefileIndexSize] up on the user mailing list.
See the thread link:http://search-hadoop.com/m/hemBv1LiN4Q1/a+question+storefileIndexSize&subj=a+question+storefileIndexSize[a question storefileIndexSize] up on the user mailing list.
Most of the time small inefficiencies don't matter all that much.
Unfortunately, this is a case where they do.
Whatever patterns are selected for ColumnFamilies, attributes, and rowkeys they could be repeated several billion times in your data.
Most of the time small inefficiencies don't matter all that much. Unfortunately, this is a case where they do.
Whatever patterns are selected for ColumnFamilies, attributes, and rowkeys they could be repeated several billion times in your data.
See <<keyvalue,keyvalue>> for more information on HBase stores data internally to see why this is important.
[[keysize.cf]]
==== Column Families
Try to keep the ColumnFamily names as small as possible, preferably one character (e.g.
"d" for data/default).
Try to keep the ColumnFamily names as small as possible, preferably one character (e.g. "d" for data/default).
See <<keyvalue,keyvalue>> for more information on HBase stores data internally to see why this is important.
See <<keyvalue>> for more information on HBase stores data internally to see why this is important.
[[keysize.attributes]]
==== Attributes
Although verbose attribute names (e.g., "myVeryImportantAttribute") are easier to read, prefer shorter attribute names (e.g., "via") to store in HBase.
Although verbose attribute names (e.g., "myVeryImportantAttribute") are easier to read, prefer shorter attribute names (e.g., "via") to store in HBase.
See <<keyvalue,keyvalue>> for more information on HBase stores data internally to see why this is important.
[[keysize.row]]
==== Rowkey Length
Keep them as short as is reasonable such that they can still be useful for required data access (e.g., Get vs.
Keep them as short as is reasonable such that they can still be useful for required data access (e.g. Get vs.
Scan). A short key that is useless for data access is not better than a longer key with better get/scan properties.
Expect tradeoffs when designing rowkeys.
Expect tradeoffs when designing rowkeys.
[[keysize.patterns]]
==== Byte Patterns
A long is 8 bytes.
You can store an unsigned number up to 18,446,744,073,709,551,615 in those eight bytes.
If you stored this number as a String -- presuming a byte per character -- you need nearly 3x the bytes.
If you stored this number as a String -- presuming a byte per character -- you need nearly 3x the bytes.
Not convinced? Below is some sample code that you can run on your own.
@ -244,7 +236,7 @@ long l = 1234567890L;
byte[] lb = Bytes.toBytes(l);
System.out.println("long bytes length: " + lb.length); // returns 8
String s = "" + l;
String s = String.valueOf(l);
byte[] sb = Bytes.toBytes(s);
System.out.println("long as string length: " + sb.length); // returns 10
@ -277,7 +269,7 @@ COLUMN CELL
The shell makes a best effort to print a string, and it this case it decided to just print the hex.
The same will happen to your row keys inside the region names.
It can be okay if you know what's being stored, but it might also be unreadable if arbitrary data can be put in the same cells.
This is the main trade-off.
This is the main trade-off.
[[reverse.timestamp]]
=== Reverse Timestamps
@ -285,33 +277,32 @@ This is the main trade-off.
.Reverse Scan API
[NOTE]
====
link:https://issues.apache.org/jira/browse/HBASE-4811[HBASE-4811] implements an API to scan a table or a range within a table in reverse, reducing the need to optimize your schema for forward or reverse scanning.
link:https://issues.apache.org/jira/browse/HBASE-4811[HBASE-4811] implements an API to scan a table or a range within a table in reverse, reducing the need to optimize your schema for forward or reverse scanning.
This feature is available in HBase 0.98 and later.
See link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setReversed%28boolean for more information.
See https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setReversed%28boolean for more information.
====
A common problem in database processing is quickly finding the most recent version of a value.
A technique using reverse timestamps as a part of the key can help greatly with a special case of this problem.
Also found in the HBase chapter of Tom White's book Hadoop: The Definitive Guide (O'Reilly), the technique involves appending (`Long.MAX_VALUE -
timestamp`) to the end of any key, e.g., [key][reverse_timestamp].
Also found in the HBase chapter of Tom White's book Hadoop: The Definitive Guide (O'Reilly), the technique involves appending (`Long.MAX_VALUE - timestamp`) to the end of any key, e.g. [key][reverse_timestamp].
The most recent value for [key] in a table can be found by performing a Scan for [key] and obtaining the first record.
Since HBase keys are in sorted order, this key sorts before any older row-keys for [key] and thus is first.
Since HBase keys are in sorted order, this key sorts before any older row-keys for [key] and thus is first.
This technique would be used instead of using <<schema.versions,schema.versions>> where the intent is to hold onto all versions "forever" (or a very long time) and at the same time quickly obtain access to any other version by using the same Scan technique.
This technique would be used instead of using <<schema.versions>> where the intent is to hold onto all versions "forever" (or a very long time) and at the same time quickly obtain access to any other version by using the same Scan technique.
[[rowkey.scope]]
=== Rowkeys and ColumnFamilies
Rowkeys are scoped to ColumnFamilies.
Thus, the same rowkey could exist in each ColumnFamily that exists in a table without collision.
Thus, the same rowkey could exist in each ColumnFamily that exists in a table without collision.
[[changing.rowkeys]]
=== Immutability of Rowkeys
Rowkeys cannot be changed.
The only way they can be "changed" in a table is if the row is deleted and then re-inserted.
This is a fairly common question on the HBase dist-list so it pays to get the rowkeys right the first time (and/or before you've inserted a lot of data).
This is a fairly common question on the HBase dist-list so it pays to get the rowkeys right the first time (and/or before you've inserted a lot of data).
[[rowkey.regionsplits]]
=== Relationship Between RowKeys and Region Splits
@ -332,21 +323,20 @@ As an example of why this is important, consider the example of using displayabl
102 102 102 102 102 102 102 102 102 102 102 102 102 102 102 102 // f
----
... (note: the lead byte is listed to the right as a comment.) Given that the first split is a '0' and the last split is an 'f', everything is great, right? Not so fast.
(note: the lead byte is listed to the right as a comment.) Given that the first split is a '0' and the last split is an 'f', everything is great, right? Not so fast.
The problem is that all the data is going to pile up in the first 2 regions and the last region thus creating a "lumpy" (and possibly "hot") region problem.
To understand why, refer to an link:http://www.asciitable.com[ASCII Table].
'0' is byte 48, and 'f' is byte 102, but there is a huge gap in byte values (bytes 58 to 96) that will _never
appear in this keyspace_ because the only values are [0-9] and [a-f]. Thus, the middle regions regions will never be used.
To make pre-spliting work with this example keyspace, a custom definition of splits (i.e., and not relying on the built-in split method) is required.
'0' is byte 48, and 'f' is byte 102, but there is a huge gap in byte values (bytes 58 to 96) that will _never appear in this keyspace_ because the only values are [0-9] and [a-f]. Thus, the middle regions regions will never be used.
To make pre-spliting work with this example keyspace, a custom definition of splits (i.e., and not relying on the built-in split method) is required.
Lesson #1: Pre-splitting tables is generally a best practice, but you need to pre-split them in such a way that all the regions are accessible in the keyspace.
While this example demonstrated the problem with a hex-key keyspace, the same problem can happen with _any_ keyspace.
Know your data.
Know your data.
Lesson #2: While generally not advisable, using hex-keys (and more generally, displayable data) can still work with pre-split tables as long as all the created regions are accessible in the keyspace.
Lesson #2: While generally not advisable, using hex-keys (and more generally, displayable data) can still work with pre-split tables as long as all the created regions are accessible in the keyspace.
To conclude this example, the following is an example of how appropriate splits can be pre-created for hex-keys:.
To conclude this example, the following is an example of how appropriate splits can be pre-created for hex-keys:.
[source,java]
----
@ -379,59 +369,58 @@ public static byte[][] getHexSplits(String startKey, String endKey, int numRegio
----
[[schema.versions]]
== Number of Versions
== Number of Versions
[[schema.versions.max]]
=== Maximum Number of Versions
The maximum number of row versions to store is configured per column family via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor].
The default for max versions is 1.
This is an important parameter because as described in <<datamodel,datamodel>> section HBase does _not_ overwrite row values, but rather stores different values per row by time (and qualifier). Excess versions are removed during major compactions.
The number of max versions may need to be increased or decreased depending on application needs.
This is an important parameter because as described in <<datamodel>> section HBase does _not_ overwrite row values, but rather stores different values per row by time (and qualifier). Excess versions are removed during major compactions.
The number of max versions may need to be increased or decreased depending on application needs.
It is not recommended setting the number of max versions to an exceedingly high level (e.g., hundreds or more) unless those old values are very dear to you because this will greatly increase StoreFile size.
It is not recommended setting the number of max versions to an exceedingly high level (e.g., hundreds or more) unless those old values are very dear to you because this will greatly increase StoreFile size.
[[schema.minversions]]
=== Minimum Number of Versions
=== Minimum Number of Versions
Like maximum number of row versions, the minimum number of row versions to keep is configured per column family via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor].
The default for min versions is 0, which means the feature is disabled.
The minimum number of row versions parameter is used together with the time-to-live parameter and can be combined with the number of row versions parameter to allow configurations such as "keep the last T minutes worth of data, at most N versions, _but keep at least M versions
around_" (where M is the value for minimum number of row versions, M<N). This parameter should only be set when time-to-live is enabled for a column family and must be less than the number of row versions.
The minimum number of row versions parameter is used together with the time-to-live parameter and can be combined with the number of row versions parameter to allow configurations such as "keep the last T minutes worth of data, at most N versions, _but keep at least M versions around_" (where M is the value for minimum number of row versions, M<N). This parameter should only be set when time-to-live is enabled for a column family and must be less than the number of row versions.
[[supported.datatypes]]
== Supported Datatypes
== Supported Datatypes
HBase supports a "bytes-in/bytes-out" interface via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html[Put] and link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Result.html[Result], so anything that can be converted to an array of bytes can be stored as a value.
Input could be strings, numbers, complex objects, or even images as long as they can rendered as bytes.
HBase supports a "bytes-in/bytes-out" interface via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html[Put] and link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Result.html[Result], so anything that can be converted to an array of bytes can be stored as a value.
Input could be strings, numbers, complex objects, or even images as long as they can rendered as bytes.
There are practical limits to the size of values (e.g., storing 10-50MB objects in HBase would probably be too much to ask); search the mailling list for conversations on this topic.
All rows in HBase conform to the <<datamodel,datamodel>>, and that includes versioning.
Take that into consideration when making your design, as well as block size for the ColumnFamily.
All rows in HBase conform to the <<datamodel>>, and that includes versioning.
Take that into consideration when making your design, as well as block size for the ColumnFamily.
=== Counters
One supported datatype that deserves special mention are "counters" (i.e., the ability to do atomic increments of numbers). See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#increment%28org.apache.hadoop.hbase.client.Increment%29[Increment] in HTable.
One supported datatype that deserves special mention are "counters" (i.e., the ability to do atomic increments of numbers). See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#increment%28org.apache.hadoop.hbase.client.Increment%29[Increment] in HTable.
Synchronization on counters are done on the RegionServer, not in the client.
Synchronization on counters are done on the RegionServer, not in the client.
[[schema.joins]]
== Joins
If you have multiple tables, don't forget to factor in the potential for <<joins,joins>> into the schema design.
If you have multiple tables, don't forget to factor in the potential for <<joins>> into the schema design.
[[ttl]]
== Time To Live (TTL)
ColumnFamilies can set a TTL length in seconds, and HBase will automatically delete rows once the expiration time is reached.
This applies to _all_ versions of a row - even the current one.
The TTL time encoded in the HBase for the row is specified in UTC.
The TTL time encoded in the HBase for the row is specified in UTC.
Store files which contains only expired rows are deleted on minor compaction.
Setting `hbase.store.delete.expired.storefile` to `false` disables this feature.
Setting link:[minimum number of versions] to other than 0 also disables this.
Setting minimum number of versions to other than 0 also disables this.
See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor] for more information.
See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor] for more information.
Recent versions of HBase also support setting time to live on a per cell basis.
See link:https://issues.apache.org/jira/browse/HBASE-10560[HBASE-10560] for more information.
@ -443,17 +432,17 @@ There are two notable differences between cell TTL handling and ColumnFamily TTL
* A cell TTLs cannot extend the effective lifetime of a cell beyond a ColumnFamily level TTL setting.
[[cf.keep.deleted]]
== Keeping Deleted Cells
== Keeping Deleted Cells
By default, delete markers extend back to the beginning of time.
Therefore, link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html[Get] or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scan] operations will not see a deleted cell (row or column), even when the Get or Scan operation indicates a time range before the delete marker was placed.
Therefore, link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html[Get] or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scan] operations will not see a deleted cell (row or column), even when the Get or Scan operation indicates a time range before the delete marker was placed.
ColumnFamilies can optionally keep deleted cells.
In this case, deleted cells can still be retrieved, as long as these operations specify a time range that ends before the timestamp of any delete that would affect the cells.
This allows for point-in-time queries even in the presence of deletes.
This allows for point-in-time queries even in the presence of deletes.
Deleted cells are still subject to TTL and there will never be more than "maximum number of versions" deleted cells.
A new "raw" scan options returns all deleted rows and the delete markers.
A new "raw" scan options returns all deleted rows and the delete markers.
.Change the Value of `KEEP_DELETED_CELLS` Using HBase Shell
====
@ -472,45 +461,43 @@ HColumnDescriptor.setKeepDeletedCells(true);
----
====
See the API documentation for link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html#KEEP_DELETED_CELLS[KEEP_DELETED_CELLS] for more information.
See the API documentation for link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html#KEEP_DELETED_CELLS[KEEP_DELETED_CELLS] for more information.
[[secondary.indexes]]
== Secondary Indexes and Alternate Query Paths
== Secondary Indexes and Alternate Query Paths
This section could also be titled "what if my table rowkey looks like _this_ but I also want to query my table like _that_." A common example on the dist-list is where a row-key is of the format "user-timestamp" but there are reporting requirements on activity across users for certain time ranges.
Thus, selecting by user is easy because it is in the lead position of the key, but time is not.
Thus, selecting by user is easy because it is in the lead position of the key, but time is not.
There is no single answer on the best way to handle this because it depends on...
There is no single answer on the best way to handle this because it depends on...
* Number of users
* Data size and data arrival rate
* Flexibility of reporting requirements (e.g., completely ad-hoc date selection vs.
pre-configured ranges)
* Desired execution speed of query (e.g., 90 seconds may be reasonable to some for an ad-hoc report, whereas it may be too long for others)
* Flexibility of reporting requirements (e.g., completely ad-hoc date selection vs. pre-configured ranges)
* Desired execution speed of query (e.g., 90 seconds may be reasonable to some for an ad-hoc report, whereas it may be too long for others)
... and solutions are also influenced by the size of the cluster and how much processing power you have to throw at the solution.
and solutions are also influenced by the size of the cluster and how much processing power you have to throw at the solution.
Common techniques are in sub-sections below.
This is a comprehensive, but not exhaustive, list of approaches.
This is a comprehensive, but not exhaustive, list of approaches.
It should not be a surprise that secondary indexes require additional cluster space and processing.
This is precisely what happens in an RDBMS because the act of creating an alternate index requires both space and processing cycles to update.
RDBMS products are more advanced in this regard to handle alternative index management out of the box.
However, HBase scales better at larger data volumes, so this is a feature trade-off.
However, HBase scales better at larger data volumes, so this is a feature trade-off.
Pay attention to <<performance,performance>> when implementing any of these approaches.
Pay attention to <<performance>> when implementing any of these approaches.
Additionally, see the David Butler response in this dist-list thread link:http://search-hadoop.com/m/nvbiBp2TDP/Stargate%252Bhbase&subj=Stargate+hbase[HBase,
mail # user - Stargate+hbase]
Additionally, see the David Butler response in this dist-list thread link:http://search-hadoop.com/m/nvbiBp2TDP/Stargate%252Bhbase&subj=Stargate+hbase[HBase, mail # user - Stargate+hbase]
[[secondary.indexes.filter]]
=== Filter Query
=== Filter Query
Depending on the case, it may be appropriate to use <<client.filter,client.filter>>.
Depending on the case, it may be appropriate to use <<client.filter>>.
In this case, no secondary index is created.
However, don't try a full-scan on a large table like this from an application (i.e., single-threaded client).
However, don't try a full-scan on a large table like this from an application (i.e., single-threaded client).
[[secondary.indexes.periodic]]
=== Periodic-Update Secondary Index
=== Periodic-Update Secondary Index
A secondary index could be created in an other table which is periodically updated via a MapReduce job.
The job could be executed intra-day, but depending on load-strategy it could still potentially be out of sync with the main data table.
@ -518,12 +505,12 @@ The job could be executed intra-day, but depending on load-strategy it could sti
See <<mapreduce.example.readwrite,mapreduce.example.readwrite>> for more information.
[[secondary.indexes.dualwrite]]
=== Dual-Write Secondary Index
=== Dual-Write Secondary Index
Another strategy is to build the secondary index while publishing data to the cluster (e.g., write to data table, write to index table). If this is approach is taken after a data table already exists, then bootstrapping will be needed for the secondary index with a MapReduce job (see <<secondary.indexes.periodic,secondary.indexes.periodic>>).
[[secondary.indexes.summary]]
=== Summary Tables
=== Summary Tables
Where time-ranges are very wide (e.g., year-long report) and where the data is voluminous, summary tables are a common approach.
These would be generated with MapReduce jobs into another table.
@ -531,29 +518,27 @@ These would be generated with MapReduce jobs into another table.
See <<mapreduce.example.summary,mapreduce.example.summary>> for more information.
[[secondary.indexes.coproc]]
=== Coprocessor Secondary Index
=== Coprocessor Secondary Index
Coprocessors act like RDBMS triggers.
These were added in 0.92.
For more information, see <<coprocessors,coprocessors>>
Coprocessors act like RDBMS triggers. These were added in 0.92.
For more information, see <<coprocessors,coprocessors>>
== Constraints
HBase currently supports 'constraints' in traditional (SQL) database parlance.
The advised usage for Constraints is in enforcing business rules for attributes in the table (eg.
make sure values are in the range 1-10). Constraints could also be used to enforce referential integrity, but this is strongly discouraged as it will dramatically decrease the write throughput of the tables where integrity checking is enabled.
Extensive documentation on using Constraints can be found at: link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/constraint[Constraint] since version 0.94.
The advised usage for Constraints is in enforcing business rules for attributes in the table (e.g. make sure values are in the range 1-10). Constraints could also be used to enforce referential integrity, but this is strongly discouraged as it will dramatically decrease the write throughput of the tables where integrity checking is enabled.
Extensive documentation on using Constraints can be found at: link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/constraint[Constraint] since version 0.94.
[[schema.casestudies]]
== Schema Design Case Studies
The following will describe some typical data ingestion use-cases with HBase, and how the rowkey design and construction can be approached.
Note: this is just an illustration of potential approaches, not an exhaustive list.
Know your data, and know your processing requirements.
Know your data, and know your processing requirements.
It is highly recommended that you read the rest of the <<schema,schema>> first, before reading these case studies.
It is highly recommended that you read the rest of the <<schema>> first, before reading these case studies.
The following case studies are described:
The following case studies are described:
* Log Data / Timeseries Data
* Log Data / Timeseries on Steroids
@ -564,21 +549,21 @@ The following case studies are described:
[[schema.casestudies.log_timeseries]]
=== Case Study - Log Data and Timeseries Data
Assume that the following data elements are being collected.
Assume that the following data elements are being collected.
* Hostname
* Timestamp
* Log event
* Value/message
We can store them in an HBase table called LOG_DATA, but what will the rowkey be? From these attributes the rowkey will be some combination of hostname, timestamp, and log-event - but what specifically?
We can store them in an HBase table called LOG_DATA, but what will the rowkey be? From these attributes the rowkey will be some combination of hostname, timestamp, and log-event - but what specifically?
[[schema.casestudies.log_timeseries.tslead]]
==== Timestamp In The Rowkey Lead Position
The rowkey `[timestamp][hostname][log-event]` suffers from the monotonically increasing rowkey problem described in <<timeseries,timeseries>>.
The rowkey `[timestamp][hostname][log-event]` suffers from the monotonically increasing rowkey problem described in <<timeseries>>.
There is another pattern frequently mentioned in the dist-lists about ``bucketing'' timestamps, by performing a mod operation on the timestamp.
There is another pattern frequently mentioned in the dist-lists about "bucketing" timestamps, by performing a mod operation on the timestamp.
If time-oriented scans are important, this could be a useful approach.
Attention must be paid to the number of buckets, because this will require the same number of scans to return results.
@ -588,7 +573,7 @@ Attention must be paid to the number of buckets, because this will require the s
long bucket = timestamp % numBuckets;
----
... to construct:
to construct:
[source]
----
@ -597,40 +582,39 @@ long bucket = timestamp % numBuckets;
----
As stated above, to select data for a particular timerange, a Scan will need to be performed for each bucket.
100 buckets, for example, will provide a wide distribution in the keyspace but it will require 100 Scans to obtain data for a single timestamp, so there are trade-offs.
100 buckets, for example, will provide a wide distribution in the keyspace but it will require 100 Scans to obtain data for a single timestamp, so there are trade-offs.
[[schema.casestudies.log_timeseries.hostlead]]
==== Host In The Rowkey Lead Position
The rowkey `[hostname][log-event][timestamp]` is a candidate if there is a large-ish number of hosts to spread the writes and reads across the keyspace.
This approach would be useful if scanning by hostname was a priority.
This approach would be useful if scanning by hostname was a priority.
[[schema.casestudies.log_timeseries.revts]]
==== Timestamp, or Reverse Timestamp?
If the most important access path is to pull most recent events, then storing the timestamps as reverse-timestamps (e.g., `timestamp = Long.MAX_VALUE
timestamp`) will create the property of being able to do a Scan on `[hostname][log-event]` to obtain the quickly obtain the most recently captured events.
If the most important access path is to pull most recent events, then storing the timestamps as reverse-timestamps (e.g., `timestamp = Long.MAX_VALUE timestamp`) will create the property of being able to do a Scan on `[hostname][log-event]` to obtain the quickly obtain the most recently captured events.
Neither approach is wrong, it just depends on what is most appropriate for the situation.
Neither approach is wrong, it just depends on what is most appropriate for the situation.
.Reverse Scan API
[NOTE]
====
link:https://issues.apache.org/jira/browse/HBASE-4811[HBASE-4811] implements an API to scan a table or a range within a table in reverse, reducing the need to optimize your schema for forward or reverse scanning.
link:https://issues.apache.org/jira/browse/HBASE-4811[HBASE-4811] implements an API to scan a table or a range within a table in reverse, reducing the need to optimize your schema for forward or reverse scanning.
This feature is available in HBase 0.98 and later.
See link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setReversed%28boolean for more information.
See https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setReversed%28boolean for more information.
====
[[schema.casestudies.log_timeseries.varkeys]]
==== Variangle Length or Fixed Length Rowkeys?
It is critical to remember that rowkeys are stamped on every column in HBase.
If the hostname is ``a'' and the event type is ``e1'' then the resulting rowkey would be quite small.
However, what if the ingested hostname is ``myserver1.mycompany.com'' and the event type is ``com.package1.subpackage2.subsubpackage3.ImportantService''?
If the hostname is `a` and the event type is `e1` then the resulting rowkey would be quite small.
However, what if the ingested hostname is `myserver1.mycompany.com` and the event type is `com.package1.subpackage2.subsubpackage3.ImportantService`?
It might make sense to use some substitution in the rowkey.
There are at least two approaches: hashed and numeric.
In the Hostname In The Rowkey Lead Position example, it might look like this:
In the Hostname In The Rowkey Lead Position example, it might look like this:
Composite Rowkey With Hashes:
@ -638,33 +622,30 @@ Composite Rowkey With Hashes:
* [MD5 hash of event-type] = 16 bytes
* [timestamp] = 8 bytes
Composite Rowkey With Numeric Substitution:
Composite Rowkey With Numeric Substitution:
For this approach another lookup table would be needed in addition to LOG_DATA, called LOG_TYPES.
The rowkey of LOG_TYPES would be:
The rowkey of LOG_TYPES would be:
* [type] (e.g., byte indicating hostname vs.
event-type)
* [type] (e.g., byte indicating hostname vs. event-type)
* [bytes] variable length bytes for raw hostname or event-type.
A column for this rowkey could be a long with an assigned number, which could be obtained by using an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#incrementColumnValue%28byte[],%20byte[],%20byte[],%20long%29[HBase
counter].
A column for this rowkey could be a long with an assigned number, which could be obtained by using an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#incrementColumnValue%28byte[],%20byte[],%20byte[],%20long%29[HBase counter].
So the resulting composite rowkey would be:
So the resulting composite rowkey would be:
* [substituted long for hostname] = 8 bytes
* [substituted long for event type] = 8 bytes
* [timestamp] = 8 bytes
In either the Hash or Numeric substitution approach, the raw values for hostname and event-type can be stored as columns.
In either the Hash or Numeric substitution approach, the raw values for hostname and event-type can be stored as columns.
[[schema.casestudies.log_steroids]]
=== Case Study - Log Data and Timeseries Data on Steroids
This effectively is the OpenTSDB approach.
What OpenTSDB does is re-write data and pack rows into columns for certain time-periods.
For a detailed explanation, see: link:http://opentsdb.net/schema.html, and link:http://www.cloudera.com/content/cloudera/en/resources/library/hbasecon/video-hbasecon-2012-lessons-learned-from-opentsdb.html[Lessons
Learned from OpenTSDB] from HBaseCon2012.
For a detailed explanation, see: link:http://opentsdb.net/schema.html, and link:http://www.cloudera.com/content/cloudera/en/resources/library/hbasecon/video-hbasecon-2012-lessons-learned-from-opentsdb.html[Lessons Learned from OpenTSDB] from HBaseCon2012.
But this is how the general concept works: data is ingested, for example, in this manner...
@ -675,52 +656,52 @@ But this is how the general concept works: data is ingested, for example, in thi
[hostname][log-event][timestamp3]
----
... with separate rowkeys for each detailed event, but is re-written like this...
with separate rowkeys for each detailed event, but is re-written like this...
----
[hostname][log-event][timerange]
----
... and each of the above events are converted into columns stored with a time-offset relative to the beginning timerange (e.g., every 5 minutes). This is obviously a very advanced processing technique, but HBase makes this possible.
and each of the above events are converted into columns stored with a time-offset relative to the beginning timerange (e.g., every 5 minutes). This is obviously a very advanced processing technique, but HBase makes this possible.
[[schema.casestudies.custorder]]
=== Case Study - Customer/Order
Assume that HBase is used to store customer and order information.
There are two core record-types being ingested: a Customer record type, and Order record type.
There are two core record-types being ingested: a Customer record type, and Order record type.
The Customer record type would include all the things that you'd typically expect:
The Customer record type would include all the things that you'd typically expect:
* Customer number
* Customer name
* Address (e.g., city, state, zip)
* Phone numbers, etc.
The Order record type would include things like:
The Order record type would include things like:
* Customer number
* Order number
* Sales date
* A series of nested objects for shipping locations and line-items (see <<schema.casestudies.custorder.obj,schema.casestudies.custorder.obj>> for details)
* A series of nested objects for shipping locations and line-items (see <<schema.casestudies.custorder.obj>> for details)
Assuming that the combination of customer number and sales order uniquely identify an order, these two attributes will compose the rowkey, and specifically a composite key such as:
Assuming that the combination of customer number and sales order uniquely identify an order, these two attributes will compose the rowkey, and specifically a composite key such as:
----
[customer number][order number]
----
... for a ORDER table.
However, there are more design decisions to make: are the _raw_ values the best choices for rowkeys?
for a ORDER table.
However, there are more design decisions to make: are the _raw_ values the best choices for rowkeys?
The same design questions in the Log Data use-case confront us here.
What is the keyspace of the customer number, and what is the format (e.g., numeric? alphanumeric?) As it is advantageous to use fixed-length keys in HBase, as well as keys that can support a reasonable spread in the keyspace, similar options appear:
What is the keyspace of the customer number, and what is the format (e.g., numeric? alphanumeric?) As it is advantageous to use fixed-length keys in HBase, as well as keys that can support a reasonable spread in the keyspace, similar options appear:
Composite Rowkey With Hashes:
Composite Rowkey With Hashes:
* [MD5 of customer number] = 16 bytes
* [MD5 of order number] = 16 bytes
Composite Numeric/Hash Combo Rowkey:
Composite Numeric/Hash Combo Rowkey:
* [substituted long for customer number] = 8 bytes
* [MD5 of order number] = 16 bytes
@ -729,20 +710,20 @@ Composite Numeric/Hash Combo Rowkey:
==== Single Table? Multiple Tables?
A traditional design approach would have separate tables for CUSTOMER and SALES.
Another option is to pack multiple record types into a single table (e.g., CUSTOMER++).
Another option is to pack multiple record types into a single table (e.g., CUSTOMER++).
Customer Record Type Rowkey:
Customer Record Type Rowkey:
* [customer-id]
* [type] = type indicating `1' for customer record type
Order Record Type Rowkey:
Order Record Type Rowkey:
* [customer-id]
* [type] = type indicating `2' for order record type
* [order]
The advantage of this particular CUSTOMER++ approach is that organizes many different record-types by customer-id (e.g., a single scan could get you everything about that customer). The disadvantage is that it's not as easy to scan for a particular record-type.
The advantage of this particular CUSTOMER++ approach is that organizes many different record-types by customer-id (e.g., a single scan could get you everything about that customer). The disadvantage is that it's not as easy to scan for a particular record-type.
[[schema.casestudies.custorder.obj]]
==== Order Object Design
@ -756,52 +737,52 @@ Order::
LineItem::
(a ShippingLocation can have multiple LineItems
... there are multiple options on storing this data.
there are multiple options on storing this data.
[[schema.casestudies.custorder.obj.norm]]
===== Completely Normalized
With this approach, there would be separate tables for ORDER, SHIPPING_LOCATION, and LINE_ITEM.
With this approach, there would be separate tables for ORDER, SHIPPING_LOCATION, and LINE_ITEM.
The ORDER table's rowkey was described above: <<schema.casestudies.custorder,schema.casestudies.custorder>>
The ORDER table's rowkey was described above: <<schema.casestudies.custorder,schema.casestudies.custorder>>
The SHIPPING_LOCATION's composite rowkey would be something like this:
The SHIPPING_LOCATION's composite rowkey would be something like this:
* [order-rowkey]
* [shipping location number] (e.g., 1st location, 2nd, etc.)
The LINE_ITEM table's composite rowkey would be something like this:
The LINE_ITEM table's composite rowkey would be something like this:
* [order-rowkey]
* [shipping location number] (e.g., 1st location, 2nd, etc.)
* [line item number] (e.g., 1st lineitem, 2nd, etc.)
Such a normalized model is likely to be the approach with an RDBMS, but that's not your only option with HBase.
The cons of such an approach is that to retrieve information about any Order, you will need:
The cons of such an approach is that to retrieve information about any Order, you will need:
* Get on the ORDER table for the Order
* Scan on the SHIPPING_LOCATION table for that order to get the ShippingLocation instances
* Scan on the LINE_ITEM for each ShippingLocation
... granted, this is what an RDBMS would do under the covers anyway, but since there are no joins in HBase you're just more aware of this fact.
granted, this is what an RDBMS would do under the covers anyway, but since there are no joins in HBase you're just more aware of this fact.
[[schema.casestudies.custorder.obj.rectype]]
===== Single Table With Record Types
With this approach, there would exist a single table ORDER that would contain
With this approach, there would exist a single table ORDER that would contain
The Order rowkey was described above: <<schema.casestudies.custorder,schema.casestudies.custorder>>
* [order-rowkey]
* [ORDER record type]
The ShippingLocation composite rowkey would be something like this:
The ShippingLocation composite rowkey would be something like this:
* [order-rowkey]
* [SHIPPING record type]
* [shipping location number] (e.g., 1st location, 2nd, etc.)
The LineItem composite rowkey would be something like this:
The LineItem composite rowkey would be something like this:
* [order-rowkey]
* [LINE record type]
@ -811,16 +792,15 @@ The LineItem composite rowkey would be something like this:
[[schema.casestudies.custorder.obj.denorm]]
===== Denormalized
A variant of the Single Table With Record Types approach is to denormalize and flatten some of the object hierarchy, such as collapsing the ShippingLocation attributes onto each LineItem instance.
A variant of the Single Table With Record Types approach is to denormalize and flatten some of the object hierarchy, such as collapsing the ShippingLocation attributes onto each LineItem instance.
The LineItem composite rowkey would be something like this:
The LineItem composite rowkey would be something like this:
* [order-rowkey]
* [LINE record type]
* [line item number] (e.g., 1st lineitem, 2nd, etc.
- care must be taken that there are unique across the entire order)
* [line item number] (e.g., 1st lineitem, 2nd, etc., care must be taken that there are unique across the entire order)
... and the LineItem columns would be something like this:
and the LineItem columns would be something like this:
* itemNumber
* quantity
@ -831,42 +811,42 @@ The LineItem composite rowkey would be something like this:
* shipToState (denormalized from ShippingLocation)
* shipToZip (denormalized from ShippingLocation)
The pros of this approach include a less complex object heirarchy, but one of the cons is that updating gets more complicated in case any of this information changes.
The pros of this approach include a less complex object hierarchy, but one of the cons is that updating gets more complicated in case any of this information changes.
[[schema.casestudies.custorder.obj.singleobj]]
===== Object BLOB
With this approach, the entire Order object graph is treated, in one way or another, as a BLOB.
For example, the ORDER table's rowkey was described above: <<schema.casestudies.custorder,schema.casestudies.custorder>>, and a single column called "order" would contain an object that could be deserialized that contained a container Order, ShippingLocations, and LineItems.
For example, the ORDER table's rowkey was described above: <<schema.casestudies.custorder,schema.casestudies.custorder>>, and a single column called "order" would contain an object that could be deserialized that contained a container Order, ShippingLocations, and LineItems.
There are many options here: JSON, XML, Java Serialization, Avro, Hadoop Writables, etc.
All of them are variants of the same approach: encode the object graph to a byte-array.
Care should be taken with this approach to ensure backward compatibilty in case the object model changes such that older persisted structures can still be read back out of HBase.
Care should be taken with this approach to ensure backward compatibilty in case the object model changes such that older persisted structures can still be read back out of HBase.
Pros are being able to manage complex object graphs with minimal I/O (e.g., a single HBase Get per Order in this example), but the cons include the aforementioned warning about backward compatiblity of serialization, language dependencies of serialization (e.g., Java Serialization only works with Java clients), the fact that you have to deserialize the entire object to get any piece of information inside the BLOB, and the difficulty in getting frameworks like Hive to work with custom objects like this.
Pros are being able to manage complex object graphs with minimal I/O (e.g., a single HBase Get per Order in this example), but the cons include the aforementioned warning about backward compatiblity of serialization, language dependencies of serialization (e.g., Java Serialization only works with Java clients), the fact that you have to deserialize the entire object to get any piece of information inside the BLOB, and the difficulty in getting frameworks like Hive to work with custom objects like this.
[[schema.smackdown]]
=== Case Study - "Tall/Wide/Middle" Schema Design Smackdown
This section will describe additional schema design questions that appear on the dist-list, specifically about tall and wide tables.
These are general guidelines and not laws - each application must consider its own needs.
These are general guidelines and not laws - each application must consider its own needs.
[[schema.smackdown.rowsversions]]
==== Rows vs. Versions
A common question is whether one should prefer rows or HBase's built-in-versioning.
The context is typically where there are "a lot" of versions of a row to be retained (e.g., where it is significantly above the HBase default of 1 max versions). The rows-approach would require storing a timestamp in some portion of the rowkey so that they would not overwite with each successive update.
The context is typically where there are "a lot" of versions of a row to be retained (e.g., where it is significantly above the HBase default of 1 max versions). The rows-approach would require storing a timestamp in some portion of the rowkey so that they would not overwite with each successive update.
Preference: Rows (generally speaking).
Preference: Rows (generally speaking).
[[schema.smackdown.rowscols]]
==== Rows vs. Columns
Another common question is whether one should prefer rows or columns.
The context is typically in extreme cases of wide tables, such as having 1 row with 1 million attributes, or 1 million rows with 1 columns apiece.
The context is typically in extreme cases of wide tables, such as having 1 row with 1 million attributes, or 1 million rows with 1 columns apiece.
Preference: Rows (generally speaking). To be clear, this guideline is in the context is in extremely wide cases, not in the standard use-case where one needs to store a few dozen or hundred columns.
But there is also a middle path between these two options, and that is "Rows as Columns."
But there is also a middle path between these two options, and that is "Rows as Columns."
[[schema.smackdown.rowsascols]]
==== Rows as Columns
@ -875,17 +855,17 @@ The middle path between Rows vs.
Columns is packing data that would be a separate row into columns, for certain rows.
OpenTSDB is the best example of this case where a single row represents a defined time-range, and then discrete events are treated as columns.
This approach is often more complex, and may require the additional complexity of re-writing your data, but has the advantage of being I/O efficient.
For an overview of this approach, see <<schema.casestudies.log_steroids,schema.casestudies.log-steroids>>.
For an overview of this approach, see <<schema.casestudies.log_steroids,schema.casestudies.log-steroids>>.
[[casestudies.schema.listdata]]
=== Case Study - List Data
The following is an exchange from the user dist-list regarding a fairly common question: how to handle per-user list data in Apache HBase.
The following is an exchange from the user dist-list regarding a fairly common question: how to handle per-user list data in Apache HBase.
*** QUESTION ***
We're looking at how to store a large amount of (per-user) list data in HBase, and we were trying to figure out what kind of access pattern made the most sense.
One option is store the majority of the data in a key, so we could have something like:
One option is store the majority of the data in a key, so we could have something like:
[source]
----
@ -905,7 +885,7 @@ The other option we had was to do this entirely using:
----
where each row would contain multiple values.
So in one case reading the first thirty values would be:
So in one case reading the first thirty values would be:
[source,java]
----
@ -913,7 +893,7 @@ So in one case reading the first thirty values would be:
scan { STARTROW => 'FixedWidthUsername' LIMIT => 30}
----
And in the second case it would be
And in the second case it would be
[source]
----
@ -923,21 +903,21 @@ get 'FixedWidthUserName\x00\x00\x00\x00'
The general usage pattern would be to read only the first 30 values of these lists, with infrequent access reading deeper into the lists.
Some users would have <= 30 total values in these lists, and some users would have millions (i.e.
power-law distribution)
power-law distribution)
The single-value format seems like it would take up more space on HBase, but would offer some improved retrieval / pagination flexibility.
Would there be any significant performance advantages to be able to paginate via gets vs paginating with scans?
Would there be any significant performance advantages to be able to paginate via gets vs paginating with scans?
My initial understanding was that doing a scan should be faster if our paging size is unknown (and caching is set appropriately), but that gets should be faster if we'll always need the same page size.
I've ended up hearing different people tell me opposite things about performance.
I assume the page sizes would be relatively consistent, so for most use cases we could guarantee that we only wanted one page of data in the fixed-page-length case.
I would also assume that we would have infrequent updates, but may have inserts into the middle of these lists (meaning we'd need to update all subsequent rows).
I would also assume that we would have infrequent updates, but may have inserts into the middle of these lists (meaning we'd need to update all subsequent rows).
Thanks for help / suggestions / follow-up questions.
Thanks for help / suggestions / follow-up questions.
*** ANSWER ***
If I understand you correctly, you're ultimately trying to store triples in the form "user, valueid, value", right? E.g., something like:
If I understand you correctly, you're ultimately trying to store triples in the form "user, valueid, value", right? E.g., something like:
[source]
----
@ -946,29 +926,29 @@ If I understand you correctly, you're ultimately trying to store triples in the
"user234, lastname, Smith"
----
(But the usernames are fixed width, and the valueids are fixed width).
(But the usernames are fixed width, and the valueids are fixed width).
And, your access pattern is along the lines of: "for user X, list the next 30 values, starting with valueid Y". Is that right? And these values should be returned sorted by valueid?
And, your access pattern is along the lines of: "for user X, list the next 30 values, starting with valueid Y". Is that right? And these values should be returned sorted by valueid?
The tl;dr version is that you should probably go with one row per user+value, and not build a complicated intra-row pagination scheme on your own unless you're really sure it is needed.
The tl;dr version is that you should probably go with one row per user+value, and not build a complicated intra-row pagination scheme on your own unless you're really sure it is needed.
Your two options mirror a common question people have when designing HBase schemas: should I go "tall" or "wide"? Your first schema is "tall": each row represents one value for one user, and so there are many rows in the table for each user; the row key is user + valueid, and there would be (presumably) a single column qualifier that means "the value". This is great if you want to scan over rows in sorted order by row key (thus my question above, about whether these ids are sorted correctly). You can start a scan at any user+valueid, read the next 30, and be done.
What you're giving up is the ability to have transactional guarantees around all the rows for one user, but it doesn't sound like you need that.
Doing it this way is generally recommended (see here link:http://hbase.apache.org/book.html#schema.smackdown).
Doing it this way is generally recommended (see here link:http://hbase.apache.org/book.html#schema.smackdown).
Your second option is "wide": you store a bunch of values in one row, using different qualifiers (where the qualifier is the valueid). The simple way to do that would be to just store ALL values for one user in a single row.
I'm guessing you jumped to the "paginated" version because you're assuming that storing millions of columns in a single row would be bad for performance, which may or may not be true; as long as you're not trying to do too much in a single request, or do things like scanning over and returning all of the cells in the row, it shouldn't be fundamentally worse.
The client has methods that allow you to get specific slices of columns.
The client has methods that allow you to get specific slices of columns.
Note that neither case fundamentally uses more disk space than the other; you're just "shifting" part of the identifying information for a value either to the left (into the row key, in option one) or to the right (into the column qualifiers in option 2). Under the covers, every key/value still stores the whole row key, and column family name.
(If this is a bit confusing, take an hour and watch Lars George's excellent video about understanding HBase schema design: link:http://www.youtube.com/watch?v=_HLoH_PgrLk).
(If this is a bit confusing, take an hour and watch Lars George's excellent video about understanding HBase schema design: link:http://www.youtube.com/watch?v=_HLoH_PgrLk).
A manually paginated version has lots more complexities, as you note, like having to keep track of how many things are in each page, re-shuffling if new values are inserted, etc.
That seems significantly more complex.
It might have some slight speed advantages (or disadvantages!) at extremely high throughput, and the only way to really know that would be to try it out.
If you don't have time to build it both ways and compare, my advice would be to start with the simplest option (one row per user+value). Start simple and iterate! :)
If you don't have time to build it both ways and compare, my advice would be to start with the simplest option (one row per user+value). Start simple and iterate! :)
[[schema.ops]]
== Operational and Performance Configuration Options
See the Performance section <<perf.schema,perf.schema>> for more information operational and performance schema design options, such as Bloom Filters, Table-configured regionsizes, compression, and blocksizes.
See the Performance section <<perf.schema,perf.schema>> for more information operational and performance schema design options, such as Bloom Filters, Table-configured regionsizes, compression, and blocksizes.

View File

@ -31,7 +31,7 @@ HBase provides mechanisms to secure various components and aspects of HBase and
== Using Secure HTTP (HTTPS) for the Web UI
A default HBase install uses insecure HTTP connections for web UIs for the master and region servers.
A default HBase install uses insecure HTTP connections for Web UIs for the master and region servers.
To enable secure HTTP (HTTPS) connections instead, set `hadoop.ssl.enabled` to `true` in _hbase-site.xml_.
This does not change the port used by the Web UI.
To change the port for the web UI for a given HBase component, configure that port's setting in hbase-site.xml.
@ -63,8 +63,7 @@ If you know how to fix this without opening a second port for HTTPS, patches are
== Secure Client Access to Apache HBase
Newer releases of Apache HBase (>= 0.92) support optional SASL authentication of clients.
See also Matteo Bertozzi's article on link:http://www.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding
User Authentication and Authorization in Apache HBase].
See also Matteo Bertozzi's article on link:http://www.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding User Authentication and Authorization in Apache HBase].
This describes how to set up Apache HBase and clients for connection to secure HBase resources.
@ -77,13 +76,13 @@ Hadoop Authentication Configuration::
Otherwise, you would be using strong authentication for HBase but not for the underlying HDFS, which would cancel out any benefit.
Kerberos KDC::
You need to have a working Kerberos KDC.
You need to have a working Kerberos KDC.
=== Server-side Configuration for Secure Operation
First, refer to <<security.prerequisites,security.prerequisites>> and ensure that your underlying HDFS configuration is secure.
Add the following to the `hbase-site.xml` file on every server machine in the cluster:
Add the following to the `hbase-site.xml` file on every server machine in the cluster:
[source,xml]
----
@ -101,13 +100,13 @@ Add the following to the `hbase-site.xml` file on every server machine in the cl
</property>
----
A full shutdown and restart of HBase service is required when deploying these configuration changes.
A full shutdown and restart of HBase service is required when deploying these configuration changes.
=== Client-side Configuration for Secure Operation
First, refer to <<security.prerequisites,security.prerequisites>> and ensure that your underlying HDFS configuration is secure.
First, refer to <<security.prerequisites>> and ensure that your underlying HDFS configuration is secure.
Add the following to the `hbase-site.xml` file on every client:
Add the following to the `hbase-site.xml` file on every client:
[source,xml]
----
@ -117,12 +116,12 @@ Add the following to the `hbase-site.xml` file on every client:
</property>
----
The client environment must be logged in to Kerberos from KDC or keytab via the `kinit` command before communication with the HBase cluster will be possible.
The client environment must be logged in to Kerberos from KDC or keytab via the `kinit` command before communication with the HBase cluster will be possible.
Be advised that if the `hbase.security.authentication` in the client- and server-side site files do not match, the client will not be able to communicate with the cluster.
Be advised that if the `hbase.security.authentication` in the client- and server-side site files do not match, the client will not be able to communicate with the cluster.
Once HBase is configured for secure RPC it is possible to optionally configure encrypted communication.
To do so, add the following to the `hbase-site.xml` file on every client:
To do so, add the following to the `hbase-site.xml` file on every client:
[source,xml]
----
@ -133,7 +132,7 @@ To do so, add the following to the `hbase-site.xml` file on every client:
----
This configuration property can also be set on a per connection basis.
Set it in the `Configuration` supplied to `HTable`:
Set it in the `Configuration` supplied to `HTable`:
[source,java]
----
@ -142,12 +141,12 @@ conf.set("hbase.rpc.protection", "privacy");
HTable table = new HTable(conf, tablename);
----
Expect a ~10% performance penalty for encrypted communication.
Expect a ~10% performance penalty for encrypted communication.
[[security.client.thrift]]
=== Client-side Configuration for Secure Operation - Thrift Gateway
Add the following to the `hbase-site.xml` file for every Thrift gateway:
Add the following to the `hbase-site.xml` file for every Thrift gateway:
[source,xml]
----
<property>
@ -161,28 +160,28 @@ Add the following to the `hbase-site.xml` file for every Thrift gateway:
You may have to put the concrete full hostname.
-->
</property>
----
----
Substitute the appropriate credential and keytab for [replaceable]_$USER_ and [replaceable]_$KEYTAB_ respectively.
Substitute the appropriate credential and keytab for _$USER_ and _$KEYTAB_ respectively.
In order to use the Thrift API principal to interact with HBase, it is also necessary to add the `hbase.thrift.kerberos.principal` to the `_acl_` table.
For example, to give the Thrift API principal, `thrift_server`, administrative access, a command such as this one will suffice:
For example, to give the Thrift API principal, `thrift_server`, administrative access, a command such as this one will suffice:
[source,sql]
----
grant 'thrift_server', 'RWCA'
----
For more information about ACLs, please see the <<hbase.accesscontrol.configuration>> section
For more information about ACLs, please see the <<hbase.accesscontrol.configuration>> section
The Thrift gateway will authenticate with HBase using the supplied credential.
No authentication will be performed by the Thrift gateway itself.
All client access via the Thrift gateway will use the Thrift gateway's credential and have its privilege.
All client access via the Thrift gateway will use the Thrift gateway's credential and have its privilege.
[[security.gateway.thrift]]
=== Configure the Thrift Gateway to Authenticate on Behalf of the Client
<<security.client.thrift,security.client.thrift>> describes how to authenticate a Thrift client to HBase using a fixed user.
<<security.client.thrift>> describes how to authenticate a Thrift client to HBase using a fixed user.
As an alternative, you can configure the Thrift gateway to authenticate to HBase on the client's behalf, and to access HBase using a proxy user.
This was implemented in link:https://issues.apache.org/jira/browse/HBASE-11349[HBASE-11349] for Thrift 1, and link:https://issues.apache.org/jira/browse/HBASE-11474[HBASE-11474] for Thrift 2.
@ -195,8 +194,8 @@ If you use framed transport, you cannot yet take advantage of this feature, beca
To enable it, do the following.
. Be sure Thrift is running in secure mode, by following the procedure described in <<security.client.thrift,security.client.thrift>>.
. Be sure that HBase is configured to allow proxy users, as described in <<security.rest.gateway,security.rest.gateway>>.
. Be sure Thrift is running in secure mode, by following the procedure described in <<security.client.thrift>>.
. Be sure that HBase is configured to allow proxy users, as described in <<security.rest.gateway>>.
. In _hbase-site.xml_ for each cluster node running a Thrift gateway, set the property `hbase.thrift.security.qop` to one of the following three values:
+
* `auth-conf` - authentication, integrity, and confidentiality checking
@ -204,14 +203,14 @@ To enable it, do the following.
* `auth` - authentication checking only
. Restart the Thrift gateway processes for the changes to take effect.
If a node is running Thrift, the output of the +jps+ command will list a `ThriftServer` process.
To stop Thrift on a node, run the command +bin/hbase-daemon.sh stop thrift+.
To start Thrift on a node, run the command +bin/hbase-daemon.sh start thrift+.
If a node is running Thrift, the output of the `jps` command will list a `ThriftServer` process.
To stop Thrift on a node, run the command `bin/hbase-daemon.sh stop thrift`.
To start Thrift on a node, run the command `bin/hbase-daemon.sh start thrift`.
[[security.gateway.thrift.doas]]
=== Configure the Thrift Gateway to Use the `doAs` Feature
<<security.gateway.thrift>> describes how to configure the Thrift gateway to authenticate to HBase on the client's behalf, and to access HBase using a proxy user. The limitation of this approach is that after the client is initialized with a particular set of credentials, it cannot change these credentials during the session session. The `doAs` feature provides a flexible way to impersonate multiple principals using the same client. This feature was implemented in link:https://issues.apache.org/jira/browse/HBASE-12640[HBASE-12640] for Thrift 1, but is currently not available for Thrift 2.
<<security.gateway.thrift>> describes how to configure the Thrift gateway to authenticate to HBase on the client's behalf, and to access HBase using a proxy user. The limitation of this approach is that after the client is initialized with a particular set of credentials, it cannot change these credentials during the session. The `doAs` feature provides a flexible way to impersonate multiple principals using the same client. This feature was implemented in link:https://issues.apache.org/jira/browse/HBASE-12640[HBASE-12640] for Thrift 1, but is currently not available for Thrift 2.
*To allow proxy users*, add the following to the _hbase-site.xml_ file for every HBase node:
@ -249,7 +248,7 @@ Take a look at the link:https://github.com/apache/hbase/blob/master/hbase-exampl
=== Client-side Configuration for Secure Operation - REST Gateway
Add the following to the `hbase-site.xml` file for every REST gateway:
Add the following to the `hbase-site.xml` file for every REST gateway:
[source,xml]
----
@ -263,24 +262,24 @@ Add the following to the `hbase-site.xml` file for every REST gateway:
</property>
----
Substitute the appropriate credential and keytab for [replaceable]_$USER_ and [replaceable]_$KEYTAB_ respectively.
Substitute the appropriate credential and keytab for _$USER_ and _$KEYTAB_ respectively.
The REST gateway will authenticate with HBase using the supplied credential.
No authentication will be performed by the REST gateway itself.
All client access via the REST gateway will use the REST gateway's credential and have its privilege.
All client access via the REST gateway will use the REST gateway's credential and have its privilege.
In order to use the REST API principal to interact with HBase, it is also necessary to add the `hbase.rest.kerberos.principal` to the `_acl_` table.
For example, to give the REST API principal, `rest_server`, administrative access, a command such as this one will suffice:
For example, to give the REST API principal, `rest_server`, administrative access, a command such as this one will suffice:
[source,sql]
----
grant 'rest_server', 'RWCA'
----
For more information about ACLs, please see the <<hbase.accesscontrol.configuration>> section
For more information about ACLs, please see the <<hbase.accesscontrol.configuration>> section
It should be possible for clients to authenticate with the HBase cluster through the REST gateway in a pass-through manner via SPEGNO HTTP authentication.
This is future work.
It should be possible for clients to authenticate with the HBase cluster through the REST gateway in a pass-through manner via SPNEGO HTTP authentication.
This is future work.
[[security.rest.gateway]]
=== REST Gateway Impersonation Configuration
@ -292,11 +291,11 @@ The actual users are unknown.
You can turn on the impersonation support.
With impersonation, the REST gateway user is a proxy user.
The HBase server knows the acutal/real user of each request.
So it can apply proper authorizations.
So it can apply proper authorizations.
To turn on REST gateway impersonation, we need to configure HBase servers (masters and region servers) to allow proxy users; configure REST gateway to enable impersonation.
To turn on REST gateway impersonation, we need to configure HBase servers (masters and region servers) to allow proxy users; configure REST gateway to enable impersonation.
To allow proxy users, add the following to the `hbase-site.xml` file for every HBase server:
To allow proxy users, add the following to the `hbase-site.xml` file for every HBase server:
[source,xml]
----
@ -314,9 +313,9 @@ To allow proxy users, add the following to the `hbase-site.xml` file for every H
</property>
----
Substitute the REST gateway proxy user for $USER, and the allowed group list for $GROUPS.
Substitute the REST gateway proxy user for _$USER_, and the allowed group list for _$GROUPS_.
To enable REST gateway impersonation, add the following to the `hbase-site.xml` file for every REST gateway.
To enable REST gateway impersonation, add the following to the `hbase-site.xml` file for every REST gateway.
[source,xml]
----
@ -334,35 +333,34 @@ To enable REST gateway impersonation, add the following to the `hbase-site.xml`
</property>
----
Substitute the keytab for HTTP for $KEYTAB.
Substitute the keytab for HTTP for _$KEYTAB_.
[[hbase.secure.simpleconfiguration]]
== Simple User Access to Apache HBase
Newer releases of Apache HBase (>= 0.92) support optional SASL authentication of clients.
See also Matteo Bertozzi's article on link:http://www.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding
User Authentication and Authorization in Apache HBase].
See also Matteo Bertozzi's article on link:http://www.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding User Authentication and Authorization in Apache HBase].
This describes how to set up Apache HBase and clients for simple user access to HBase resources.
=== Simple Versus Secure Access
=== Simple versus Secure Access
The following section shows how to set up simple user access.
Simple user access is not a secure method of operating HBase.
This method is used to prevent users from making mistakes.
It can be used to mimic the Access Control using on a development system without having to set up Kerberos.
It can be used to mimic the Access Control using on a development system without having to set up Kerberos.
This method is not used to prevent malicious or hacking attempts.
To make HBase secure against these types of attacks, you must configure HBase for secure operation.
Refer to the section link:[Secure Client Access to HBase] and complete all of the steps described there.
Refer to the section <<hbase.secure.configuration>> and complete all of the steps described there.
=== Prerequisites
None
None
=== Server-side Configuration for Simple User Access Operation
Add the following to the `hbase-site.xml` file on every server machine in the cluster:
Add the following to the `hbase-site.xml` file on every server machine in the cluster:
[source,xml]
----
@ -388,7 +386,7 @@ Add the following to the `hbase-site.xml` file on every server machine in the cl
</property>
----
For 0.94, add the following to the `hbase-site.xml` file on every server machine in the cluster:
For 0.94, add the following to the `hbase-site.xml` file on every server machine in the cluster:
[source,xml]
----
@ -406,11 +404,11 @@ For 0.94, add the following to the `hbase-site.xml` file on every server machine
</property>
----
A full shutdown and restart of HBase service is required when deploying these configuration changes.
A full shutdown and restart of HBase service is required when deploying these configuration changes.
=== Client-side Configuration for Simple User Access Operation
Add the following to the `hbase-site.xml` file on every client:
Add the following to the `hbase-site.xml` file on every client:
[source,xml]
----
@ -420,7 +418,7 @@ Add the following to the `hbase-site.xml` file on every client:
</property>
----
For 0.94, add the following to the `hbase-site.xml` file on every server machine in the cluster:
For 0.94, add the following to the `hbase-site.xml` file on every server machine in the cluster:
[source,xml]
----
@ -430,42 +428,42 @@ For 0.94, add the following to the `hbase-site.xml` file on every server machine
</property>
----
Be advised that if the `hbase.security.authentication` in the client- and server-side site files do not match, the client will not be able to communicate with the cluster.
Be advised that if the `hbase.security.authentication` in the client- and server-side site files do not match, the client will not be able to communicate with the cluster.
==== Client-side Configuration for Simple User Access Operation - Thrift Gateway
The Thrift gateway user will need access.
For example, to give the Thrift API user, `thrift_server`, administrative access, a command such as this one will suffice:
For example, to give the Thrift API user, `thrift_server`, administrative access, a command such as this one will suffice:
[source,sql]
----
grant 'thrift_server', 'RWCA'
----
For more information about ACLs, please see the link:[Access Control] section
For more information about ACLs, please see the <<hbase.accesscontrol.configuration>> section
The Thrift gateway will authenticate with HBase using the supplied credential.
No authentication will be performed by the Thrift gateway itself.
All client access via the Thrift gateway will use the Thrift gateway's credential and have its privilege.
All client access via the Thrift gateway will use the Thrift gateway's credential and have its privilege.
==== Client-side Configuration for Simple User Access Operation - REST Gateway
The REST gateway will authenticate with HBase using the supplied credential.
No authentication will be performed by the REST gateway itself.
All client access via the REST gateway will use the REST gateway's credential and have its privilege.
All client access via the REST gateway will use the REST gateway's credential and have its privilege.
The REST gateway user will need access.
For example, to give the REST API user, `rest_server`, administrative access, a command such as this one will suffice:
For example, to give the REST API user, `rest_server`, administrative access, a command such as this one will suffice:
[source,sql]
----
grant 'rest_server', 'RWCA'
----
For more information about ACLs, please see the link:[Access Control] section
For more information about ACLs, please see the <<hbase.accesscontrol.configuration>> section
It should be possible for clients to authenticate with the HBase cluster through the REST gateway in a pass-through manner via SPEGNO HTTP authentication.
This is future work.
It should be possible for clients to authenticate with the HBase cluster through the REST gateway in a pass-through manner via SPNEGO HTTP authentication.
This is future work.
== Securing Access To Your Data
@ -496,8 +494,9 @@ When copying keys, configuration files, or other files containing sensitive stri
====
.Procedure: Basic Server-Side Configuration
. Enable HFile v3, by setting +hfile.format.version +to 3 in _hbase-site.xml_.
This is the default for HBase 1.0 and newer. +
. Enable HFile v3, by setting `hfile.format.version` to 3 in _hbase-site.xml_.
This is the default for HBase 1.0 and newer.
+
[source,xml]
----
<property>
@ -506,7 +505,7 @@ When copying keys, configuration files, or other files containing sensitive stri
</property>
----
. Enable SASL and Kerberos authentication for RPC and ZooKeeper, as described in <<security.prerequisites,security.prerequisites>> and <<zk.sasl.auth,zk.sasl.auth>>.
. Enable SASL and Kerberos authentication for RPC and ZooKeeper, as described in <<security.prerequisites,security.prerequisites>> and <<zk.sasl.auth>>.
[[hbase.tags]]
=== Tags
@ -528,7 +527,7 @@ You can enable or disable tag encoding at the level of the column family, and it
Use the `HColumnDescriptor#setCompressionTags(boolean compressTags)` method to manage encoding settings on a column family.
You also need to enable the DataBlockEncoder for the column family, for encoding of tags to take effect.
You can enable compression of each tag in the WAL, if WAL compression is also enabled, by setting the value of +hbase.regionserver.wal.tags.enablecompression+ to `true` in _hbase-site.xml_.
You can enable compression of each tag in the WAL, if WAL compression is also enabled, by setting the value of `hbase.regionserver.wal.tags.enablecompression` to `true` in _hbase-site.xml_.
Tag compression uses dictionary encoding.
Tag compression is not supported when using WAL encryption.
@ -541,8 +540,7 @@ Tag compression is not supported when using WAL encryption.
ACLs in HBase are based upon a user's membership in or exclusion from groups, and a given group's permissions to access a given resource.
ACLs are implemented as a coprocessor called AccessController.
HBase does not maintain a private group mapping, but relies on a [firstterm]_Hadoop
group mapper_, which maps between entities in a directory such as LDAP or Active Directory, and HBase users.
HBase does not maintain a private group mapping, but relies on a [firstterm]_Hadoop group mapper_, which maps between entities in a directory such as LDAP or Active Directory, and HBase users.
Any supported Hadoop group mapper will work.
Users are then granted specific permissions (Read, Write, Execute, Create, Admin) against resources (global, namespaces, tables, cells, or endpoints).
@ -555,21 +553,21 @@ No distinction is made between an insert (new record) and update (of existing re
HBase access levels are granted independently of each other and allow for different types of operations at a given scope.
* Read \(R) - can read data at the given scope
* +Write (W)+ - can write data at the given scope
* +Execute (X)+ - can execute coprocessor endpoints at the given scope
* +Create (C)+ - can create tables or drop tables (even those they did not create) at the given scope
* +Admin (A)+ - can perform cluster operations such as balancing the cluster or assigning regions at the given scope
* _Read \(R)_ - can read data at the given scope
* _Write (W)_ - can write data at the given scope
* _Execute (X)_ - can execute coprocessor endpoints at the given scope
* _Create \(C)_ - can create tables or drop tables (even those they did not create) at the given scope
* _Admin (A)_ - can perform cluster operations such as balancing the cluster or assigning regions at the given scope
The possible scopes are:
* +Superuser+ - superusers can perform any operation available in HBase, to any resource.
* _Superuser_ - superusers can perform any operation available in HBase, to any resource.
The user who runs HBase on your cluster is a superuser, as are any principals assigned to the configuration property `hbase.superuser` in _hbase-site.xml_ on the HMaster.
* +Global+ - permissions granted at _global_ scope allow the admin to operate on all tables of the cluster.
* +Namespace+ - permissions granted at _namespace_ scope apply to all tables within a given namespace.
* +Table+ - permissions granted at _table_ scope apply to data or metadata within a given table.
* +ColumnFamily+ - permissions granted at _ColumnFamily_ scope apply to cells within that ColumnFamily.
* +Cell+ - permissions granted at _cell_ scope apply to that exact cell coordinate (key, value, timestamp). This allows for policy evolution along with data.
* _Global_ - permissions granted at _global_ scope allow the admin to operate on all tables of the cluster.
* _Namespace_ - permissions granted at _namespace_ scope apply to all tables within a given namespace.
* _Table_ - permissions granted at _table_ scope apply to data or metadata within a given table.
* _ColumnFamily_ - permissions granted at _ColumnFamily_ scope apply to cells within that ColumnFamily.
* _Cell_ - permissions granted at _cell_ scope apply to that exact cell coordinate (key, value, timestamp). This allows for policy evolution along with data.
+
To change an ACL on a specific cell, write an updated cell with new ACL to the precise coordinates of the original.
+
@ -587,12 +585,11 @@ In a production environment, it is useful to think of access levels in terms of
The following list describes appropriate access levels for some common types of HBase users.
It is important not to grant more access than is required for a given user to perform their required tasks.
* Superusers - In a production system, only the HBase user should have superuser access.
* _Superusers_ - In a production system, only the HBase user should have superuser access.
In a development environment, an administrator may need superuser access in order to quickly control and manage the cluster.
However, this type of administrator should usually be a Global Admin rather than a superuser.
* Global Admins - A global admin can perform tasks and access every table in HBase.
* _Global Admins_ - A global admin can perform tasks and access every table in HBase.
In a typical production environment, an admin should not have Read or Write permissions to data within tables.
+
* A global admin with Admin permissions can perform cluster-wide operations on the cluster, such as balancing, assigning or unassigning regions, or calling an explicit major compaction.
This is an operations role.
* A global admin with Create permissions can create or drop any table within HBase.
@ -602,20 +599,20 @@ In a production environment, it is likely that different users will have only on
+
[WARNING]
====
In the current implementation, a Global Admin with `Admin` permission can grant himself `Read` and `Write` permissions on a table and gain access to that table's data.
In the current implementation, a Global Admin with `Admin` permission can grant himself `Read` and `Write` permissions on a table and gain access to that table's data.
For this reason, only grant `Global Admin` permissions to trusted user who actually need them.
Also be aware that a `Global Admin` with `Create` permission can perform a `Put` operation on the ACL table, simulating a `grant` or `revoke` and circumventing the authorization check for `Global Admin` permissions.
Also be aware that a `Global Admin` with `Create` permission can perform a `Put` operation on the ACL table, simulating a `grant` or `revoke` and circumventing the authorization check for `Global Admin` permissions.
Due to these issues, be cautious with granting `Global Admin` privileges.
Due to these issues, be cautious with granting `Global Admin` privileges.
====
* +Namespace Admins+ - a namespace admin with `Create` permissions can create or drop tables within that namespace, and take and restore snapshots.
* _Namespace Admins_ - a namespace admin with `Create` permissions can create or drop tables within that namespace, and take and restore snapshots.
A namespace admin with `Admin` permissions can perform operations such as splits or major compactions on tables within that namespace.
* +Table Admins+ - A table admin can perform administrative operations only on that table.
* _Table Admins_ - A table admin can perform administrative operations only on that table.
A table admin with `Create` permissions can create snapshots from that table or restore that table from a snapshot.
A table admin with `Admin` permissions can perform operations such as splits or major compactions on that table.
* +Users+ - Users can read or write data, or both.
* _Users_ - Users can read or write data, or both.
Users can also execute coprocessor endpoints, if given `Executable` permissions.
.Real-World Example of Access Levels
@ -625,17 +622,16 @@ Due to these issues, be cautious with granting `Global Admin` p
| Scope
| Permissions
| Description
| Senior Administrator
| Global
| Access, Create
| Manages the cluster and gives access to Junior
Administrators.
| Manages the cluster and gives access to Junior Administrators.
| Junior Administrator
| Global
| Create
| Creates tables and gives access to Table
Administrators.
| Creates tables and gives access to Table Administrators.
| Table Administrator
| Table
@ -650,8 +646,7 @@ Due to these issues, be cautious with granting `Global Admin` p
| Web Application
| Table
| Read, Write
| Puts data into HBase and uses HBase data to perform
operations.
| Puts data into HBase and uses HBase data to perform operations.
|===
.ACL Matrix
@ -659,7 +654,7 @@ For more details on how ACLs map to specific HBase operations and tasks, see <<a
===== Implementation Details
Cell-level ACLs are implemented using tags (see <<hbase.tags,hbase.tags>>). In order to use cell-level ACLs, you must be using HFile v3 and HBase 0.98 or newer.
Cell-level ACLs are implemented using tags (see <<hbase.tags>>). In order to use cell-level ACLs, you must be using HFile v3 and HBase 0.98 or newer.
. Files created by HBase are owned by the operating system user running the HBase process.
To interact with HBase files, you should use the API or bulk load facility.
@ -670,13 +665,12 @@ Cell-level ACLs are implemented using tags (see <<hbase.tags,hbase.tags>>). In o
===== Server-Side Configuration
. As a prerequisite, perform the steps in <<security.data.basic.server.side,security.data.basic.server.side>>.
. As a prerequisite, perform the steps in <<security.data.basic.server.side>>.
. Install and configure the AccessController coprocessor, by setting the following properties in _hbase-site.xml_.
These properties take a list of classes.
These properties take a list of classes.
+
NOTE: If you use the AccessController along with the VisibilityController, the AccessController must come first in the list, because with both components active, the VisibilityController will delegate access control on its system tables to the AccessController.
For an example of using both together, see <<security.example.config,security.example.config>>.
For an example of using both together, see <<security.example.config>>.
+
[source,xml]
----
@ -698,7 +692,7 @@ For an example of using both together, see <<security.example.config,security.ex
</property>
----
+
Optionally, you can enable transport security, by setting +hbase.rpc.protection+ to `auth-conf`.
Optionally, you can enable transport security, by setting `hbase.rpc.protection` to `auth-conf`.
This requires HBase 0.98.4 or newer.
. Set up the Hadoop group mapper in the Hadoop namenode's _core-site.xml_.
@ -756,11 +750,11 @@ This requires HBase 0.98.4 or newer.
. Optionally, enable the early-out evaluation strategy.
Prior to HBase 0.98.0, if a user was not granted access to a column family, or at least a column qualifier, an AccessDeniedException would be thrown.
HBase 0.98.0 removed this exception in order to allow cell-level exceptional grants.
To restore the old behavior in HBase 0.98.0-0.98.6, set +hbase.security.access.early_out+ to `true` in _hbase-site.xml_.
To restore the old behavior in HBase 0.98.0-0.98.6, set `hbase.security.access.early_out` to `true` in _hbase-site.xml_.
In HBase 0.98.6, the default has been returned to `true`.
. Distribute your configuration and restart your cluster for changes to take effect.
. To test your configuration, log into HBase Shell as a given user and use the +whoami+ command to report the groups your user is part of.
In this example, the user is reported as being a member of the `services` group.
. To test your configuration, log into HBase Shell as a given user and use the `whoami` command to report the groups your user is part of.
In this example, the user is reported as being a member of the `services` group.
+
----
hbase> whoami
@ -798,7 +792,7 @@ grant 'user', 'RWXCA', 'TABLE', 'CF', 'CQ'
----
+
Groups and users are granted access in the same way, but groups are prefixed with an `@` symbol.
In the same way, tables and namespaces are specified in the same way, but namespaces are prefixed with an `@` symbol.
In the same way, tables and namespaces are specified in the same way, but namespaces are prefixed with an `@` symbol.
+
It is also possible to grant multiple permissions against the same resource in a single statement, as in this example.
The first sub-clause maps users to ACLs and the second sub-clause specifies the resource.
@ -853,9 +847,9 @@ grant <table>, \
{ <scanner-specification> }
----
+
* [replaceable]_<user-or-group>_ is the user or group name, prefixed with `@` in the case of a group.
* [replaceable]_<permissions>_ is a string containing any or all of "RWXCA", though only R and W are meaningful at cell scope.
* [replaceable]_<scanner-specification>_ is the scanner specification syntax and conventions used by the 'scan' shell command.
* _<user-or-group>_ is the user or group name, prefixed with `@` in the case of a group.
* _<permissions>_ is a string containing any or all of "RWXCA", though only R and W are meaningful at cell scope.
* _<scanner-specification>_ is the scanner specification syntax and conventions used by the 'scan' shell command.
For some examples of scanner specifications, issue the following HBase Shell command.
+
----
@ -920,9 +914,9 @@ put.setACL(“user1”, new Permission(Permission.Action.READ))
. Revoking Access Control From a Namespace, Table, Column Family, or Cell
+
The +revoke+ command and API are twins of the grant command and API, and the syntax is exactly the same.
The `revoke` command and API are twins of the grant command and API, and the syntax is exactly the same.
The only exception is that you cannot revoke permissions at the cell level.
You can only revoke access that has previously been granted, and a +revoke+ statement is not the same thing as explicit denial to a resource.
You can only revoke access that has previously been granted, and a `revoke` statement is not the same thing as explicit denial to a resource.
+
NOTE: HBase Shell support for granting and revoking access is for testing and verification support, and should not be employed for production use because it won't apply the permissions to cells that don't exist yet.
The correct way to apply cell-level permissions is to do so in the application code when storing the values.
@ -976,12 +970,12 @@ public static void verifyAllowed(User user, AccessTestAction action, int count)
if (obj != null && obj instanceof List&lt;?&gt;) {
List&lt;?&gt; results = (List&lt;?&gt;) obj;
if (results != null && results.isEmpty()) {
fail("Empty non null results from action for user '" + user.getShortName() + "'");
fail("Empty non null results from action for user '" ` user.getShortName() ` "'");
}
assertEquals(count, results.size());
}
} catch (AccessDeniedException ade) {
fail("Expected action to pass for user '" + user.getShortName() + "' but was denied");
fail("Expected action to pass for user '" ` user.getShortName() ` "' but was denied");
}
}
----
@ -1000,25 +994,25 @@ Visibility labels have no meaning on their own, and may be used to denote sensit
If a user's labels do not match a cell's label or expression, the user is denied access to the cell.
In HBase 0.98.6 and newer, UTF-8 encoding is supported for visibility labels and expressions.
When creating labels using the `addLabels(conf, labels)` method provided by the `org.apache.hadoop.hbase.security.visibility.VisibilityClient` class and passing labels in Authorizations via Scan or Get, labels can contain UTF-8 characters, as well as the logical operators normally used in visibility labels, with normal Java notations, without needing any escaping method.
When creating labels using the `addLabels(conf, labels)` method provided by the `org.apache.hadoop.hbase.security.visibility.VisibilityClient` class and passing labels in Authorizations via Scan or Get, labels can contain UTF-8 characters, as well as the logical operators normally used in visibility labels, with normal Java notations, without needing any escaping method.
However, when you pass a CellVisibility expression via a Mutation, you must enclose the expression with the `CellVisibility.quote()` method if you use UTF-8 characters or logical operators.
See `TestExpressionParser` and the source file _hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java_.
See `TestExpressionParser` and the source file _hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java_.
A user adds visibility expressions to a cell during a Put operation.
In the default configuration, the user does not need to access to a label in order to label cells with it.
This behavior is controlled by the configuration option +hbase.security.visibility.mutations.checkauths+.
In the default configuration, the user does not need to have access to a label in order to label cells with it.
This behavior is controlled by the configuration option `hbase.security.visibility.mutations.checkauths`.
If you set this option to `true`, the labels the user is modifying as part of the mutation must be associated with the user, or the mutation will fail.
Whether a user is authorized to read a labelled cell is determined during a Get or Scan, and results which the user is not allowed to read are filtered out.
This incurs the same I/O penalty as if the results were returned, but reduces load on the network.
Visibility labels can also be specified during Delete operations.
For details about visibility labels and Deletes, see link:https://issues.apache.org/jira/browse/HBASE-10885[HBASE-10885].
For details about visibility labels and Deletes, see link:https://issues.apache.org/jira/browse/HBASE-10885[HBASE-10885].
The user's effective label set is built in the RPC context when a request is first received by the RegionServer.
The way that users are associated with labels is pluggable.
The default plugin passes through labels specified in Authorizations added to the Get or Scan and checks those against the calling user's authenticated labels list.
When the client passes labels for which the user is not authenticated, the default plugin drops them.
You can pass a subset of user authenticated labels via the `Get#setAuthorizations(Authorizations(String,...))` and `Scan#setAuthorizations(Authorizations(String,...));` methods.
You can pass a subset of user authenticated labels via the `Get#setAuthorizations(Authorizations(String,...))` and `Scan#setAuthorizations(Authorizations(String,...));` methods.
Visibility label access checking is performed by the VisibilityController coprocessor.
You can use interface `VisibilityLabelService` to provide a custom implementation and/or control the way that visibility labels are stored with cells.
@ -1026,29 +1020,32 @@ See the source file _hbase-server/src/test/java/org/apache/hadoop/hbase/security
Visibility labels can be used in conjunction with ACLs.
NOTE: The labels have to be explicitly defined before they can be used in visibility labels. See below for an example of how this can be done.
NOTE: There is currently no way to determine which labels have been applied to a cell. See link:https://issues.apache.org/jira/browse/HBASE-12470[HBASE-12470] for details.
NOTE: Visibility labels are not currently applied for superusers.
.Examples of Visibility Expressions
[cols="l,1", options="header"]
|===
| Expression
| Interpretation
| fulltime
| Allow accesss to users associated with the
fulltime label.
| Allow accesss to users associated with the fulltime label.
| !public
| Allow access to users not associated with the
public label.
| Allow access to users not associated with the public label.
| ( secret \| topsecret ) & !probationary
| Allow access to users associated with either the
secret or topsecret label and not
associated with the probationary label.
| Allow access to users associated with either the secret or topsecret label and not associated with the probationary label.
|===
==== Server-Side Configuration
. As a prerequisite, perform the steps in <<security.data.basic.server.side,security.data.basic.server.side>>.
. As a prerequisite, perform the steps in <<security.data.basic.server.side>>.
. Install and configure the VisibilityController coprocessor by setting the following properties in _hbase-site.xml_.
These properties take a list of class names.
+
@ -1070,7 +1067,7 @@ NOTE: If you use the AccessController and VisibilityController coprocessors toge
+
By default, users can label cells with any label, including labels they are not associated with, which means that a user can Put data that he cannot read.
For example, a user could label a cell with the (hypothetical) 'topsecret' label even if the user is not associated with that label.
If you only want users to be able to label cells with labels they are associated with, set +hbase.security.visibility.mutations.checkauths+ to `true`.
If you only want users to be able to label cells with labels they are associated with, set `hbase.security.visibility.mutations.checkauths` to `true`.
In that case, the mutation will fail if it makes use of labels the user is not associated with.
. Distribute your configuration and restart your cluster for changes to take effect.
@ -1104,10 +1101,8 @@ hbase> add_labels [ 'admin', 'service', 'developer', 'test' ]
====
[source,java]
----
public static void addLabels() throws Exception {
PrivilegedExceptionAction<VisibilityLabelsResponse> action =
new PrivilegedExceptionAction<VisibilityLabelsResponse>() {
PrivilegedExceptionAction<VisibilityLabelsResponse> action = new PrivilegedExceptionAction<VisibilityLabelsResponse>() {
public VisibilityLabelsResponse run() throws Exception {
String[] labels = { SECRET, TOPSECRET, CONFIDENTIAL, PUBLIC, PRIVATE, COPYRIGHT, ACCENT,
UNICODE_VIS_TAG, UC1, UC2 };
@ -1145,7 +1140,6 @@ hbase> set_auths 'qa', [ 'test', 'developer' ]
====
[source,java]
----
public void testSetAndGetUserAuths() throws Throwable {
final String user = "user1";
PrivilegedExceptionAction<Void> action = new PrivilegedExceptionAction<Void>() {
@ -1182,7 +1176,6 @@ hbase> clear_auths 'qa', [ 'test', 'developer' ]
====
[source,java]
----
...
auths = new String[] { SECRET, PUBLIC, CONFIDENTIAL };
VisibilityLabelsResponse response = null;
@ -1190,7 +1183,8 @@ try {
response = VisibilityClient.clearAuths(conf, auths, user);
} catch (Throwable e) {
fail("Should not have failed");
...
...
}
----
====
@ -1202,18 +1196,15 @@ The label is associated with a given version of the cell.
.HBase Shell
====
----
hbase> set_visibility 'user', 'admin|service|developer', \
{ COLUMNS => 'i' }
hbase> set_visibility 'user', 'admin|service|developer', { COLUMNS => 'i' }
----
----
hbase> set_visibility 'user', 'admin|service', \
{ COLUMNS => ' pii' }
hbase> set_visibility 'user', 'admin|service', { COLUMNS => 'pii' }
----
----
hbase> COLUMNS => [ 'i', 'pii' ], \
FILTER => "(PrefixFilter ('test'))" }
hbase> set_visibility 'user', 'test', { COLUMNS => [ 'i', 'pii' ], FILTER => "(PrefixFilter ('test'))" }
----
====
+
@ -1257,7 +1248,7 @@ You can also configure a set of `ScanLabelGenerators` to be used by the system,
==== Replicating Visibility Tags as Strings
As mentioned in the above sections, the interface `VisibilityLabelService` could be used to implement a different way of storing the visibility expressions in the cells. Clusters with replication enabled also must replicate the visibility expressions to the peer cluster. If `DefaultVisibilityLabelServiceImpl` is used as the implementation for `VisibilityLabelService`, all the visibility expression are converted to the corresponding expression based on the ordinals for each visibility label stored in the labels table. During replication, visible cellsare also replicated with the ordinal-based expression intact. The peer cluster may not have the same `labels` table with the same ordinal mapping for the visibility labels. In that case, replicating the ordinals makes no sense. It would be better if the replication occurred with the visibility expressions transmitted as strings. To replicate the visibility expression as strings to the peer cluster, create a `RegionServerObserver` configuration which works based on the implementation of the `VisibilityLabelService` interface. The configuration below enables replication of visibility expressions to peer clusters as strings. See link:https://issues.apache.org/jira/browse/HBASE-11639[HBASE-11639] for more details.
As mentioned in the above sections, the interface `VisibilityLabelService` could be used to implement a different way of storing the visibility expressions in the cells. Clusters with replication enabled also must replicate the visibility expressions to the peer cluster. If `DefaultVisibilityLabelServiceImpl` is used as the implementation for `VisibilityLabelService`, all the visibility expression are converted to the corresponding expression based on the ordinals for each visibility label stored in the labels table. During replication, visible cells are also replicated with the ordinal-based expression intact. The peer cluster may not have the same `labels` table with the same ordinal mapping for the visibility labels. In that case, replicating the ordinals makes no sense. It would be better if the replication occurred with the visibility expressions transmitted as strings. To replicate the visibility expression as strings to the peer cluster, create a `RegionServerObserver` configuration which works based on the implementation of the `VisibilityLabelService` interface. The configuration below enables replication of visibility expressions to peer clusters as strings. See link:https://issues.apache.org/jira/browse/HBASE-11639[HBASE-11639] for more details.
[source,xml]
----
@ -1286,7 +1277,7 @@ The master key may be stored on the cluster servers, protected by a secure KeySt
This master key is resolved as needed by HBase processes through the configured key provider.
Next, encryption use can be specified in the schema, per column family, by creating or modifying a column descriptor to include two additional attributes: the name of the encryption algorithm to use (currently only "AES" is supported), and optionally, a data key wrapped (encrypted) with the cluster master key.
If a data key is not explictly configured for a ColumnFamily, HBase will create a random data key per HFile.
If a data key is not explicitly configured for a ColumnFamily, HBase will create a random data key per HFile.
This provides an incremental improvement in security over the alternative.
Unless you need to supply an explicit data key, such as in a case where you are generating encrypted HFiles for bulk import with a given data key, only specify the encryption algorithm in the ColumnFamily schema metadata and let HBase create data keys on demand.
Per Column Family keys facilitate low impact incremental key rotation and reduce the scope of any external leak of key material.
@ -1338,23 +1329,23 @@ In the example below, replace [replaceable]_****_ with the password.
[source,xml]
----
<property>
<name>hbase.crypto.keyprovider</name>
<value>org.apache.hadoop.hbase.io.crypto.KeyStoreKeyProvider</value>
<name>hbase.crypto.keyprovider</name>
<value>org.apache.hadoop.hbase.io.crypto.KeyStoreKeyProvider</value>
</property>
<property>
<name>hbase.crypto.keyprovider.parameters</name>
<value>jceks:///path/to/hbase/conf/hbase.jks?password=****</value>
<name>hbase.crypto.keyprovider.parameters</name>
<value>jceks:///path/to/hbase/conf/hbase.jks?password=****</value>
</property>
----
+
By default, the HBase service account name will be used to resolve the cluster master key.
However, you can store it with an arbitrary alias (in the +keytool+ command). In that case, set the following property to the alias you used.
However, you can store it with an arbitrary alias (in the `keytool` command). In that case, set the following property to the alias you used.
+
[source,xml]
----
<property>
<name>hbase.crypto.master.key.name</name>
<value>my-alias</value>
<name>hbase.crypto.master.key.name</name>
<value>my-alias</value>
</property>
----
+
@ -1365,24 +1356,22 @@ For previous versions, set the following property in your _hbase-site.xml_
[source,xml]
----
<property>
<name>hfile.format.version</name>
<value>3</value>
<name>hfile.format.version</name>
<value>3</value>
</property>
----
+
Optionally, you can use a different cipher provider, either a Java Cryptography Encryption (JCE) algorithm provider or a custom HBase cipher implementation.
Optionally, you can use a different cipher provider, either a Java Cryptography Encryption (JCE) algorithm provider or a custom HBase cipher implementation.
+
* JCE:
+
* Install a signed JCE provider (supporting ``AES/CTR/NoPadding'' mode with 128 bit keys)
* Add it with highest preference to the JCE site configuration file _$JAVA_HOME/lib/security/java.security_.
* Update +hbase.crypto.algorithm.aes.provider+ and +hbase.crypto.algorithm.rng.provider+ options in _hbase-site.xml_.
* JCE:
** Install a signed JCE provider (supporting `AES/CTR/NoPadding` mode with 128 bit keys)
** Add it with highest preference to the JCE site configuration file _$JAVA_HOME/lib/security/java.security_.
** Update `hbase.crypto.algorithm.aes.provider` and `hbase.crypto.algorithm.rng.provider` options in [path]_hbase-site.xml_.
* Custom HBase Cipher:
+
* Implement `org.apache.hadoop.hbase.io.crypto.CipherProvider`.
* Add the implementation to the server classpath.
* Update +hbase.crypto.cipherprovider+ in _hbase-site.xml_.
* Custom HBase Cipher:
** Implement `org.apache.hadoop.hbase.io.crypto.CipherProvider`.
** Add the implementation to the server classpath.
** Update `hbase.crypto.cipherprovider` in _hbase-site.xml_.
. Configure WAL encryption.
@ -1452,10 +1441,12 @@ Rotate the Master Key::
[[hbase.secure.bulkload]]
=== Secure Bulk Load
Bulk loading in secure mode is a bit more involved than normal setup, since the client has to transfer the ownership of the files generated from the mapreduce job to HBase.
Secure bulk loading is implemented by a coprocessor, named link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.html[SecureBulkLoadEndpoint], which uses a staging directory configured by the configuration property +hbase.bulkload.staging.dir+, which defaults to _/tmp/hbase-staging/_.
Bulk loading in secure mode is a bit more involved than normal setup, since the client has to transfer the ownership of the files generated from the MapReduce job to HBase.
Secure bulk loading is implemented by a coprocessor, named link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.html[SecureBulkLoadEndpoint], which uses a staging directory configured by the configuration property `hbase.bulkload.staging.dir`, which defaults to _/tmp/hbase-staging/_.
* .Secure Bulk Load AlgorithmOne time only, create a staging directory which is world-traversable and owned by the user which runs HBase (mode 711, or `rwx--x--x`). A listing of this directory will look similar to the following:
.Secure Bulk Load Algorithm
* One time only, create a staging directory which is world-traversable and owned by the user which runs HBase (mode 711, or `rwx--x--x`). A listing of this directory will look similar to the following:
+
[source,bash]
----
@ -1468,7 +1459,7 @@ drwx--x--x 2 hbase hbase 68 3 Sep 14:54 /tmp/hbase-staging
* Internally, HBase creates a secret staging directory which is globally readable/writable (`-rwxrwxrwx, 777`). For example, _/tmp/hbase-staging/averylongandrandomdirectoryname_.
The name and location of this directory is not exposed to the user.
HBase manages creation and deletion of this directory.
* The user makes the data world-readable and world-writable, moves it into the random staging directory, then calls the `SecureBulkLoadClient#bulkLoadHFiles` method.
* The user makes the data world-readable and world-writable, moves it into the random staging directory, then calls the `SecureBulkLoadClient#bulkLoadHFiles` method.
The strength of the security lies in the length and randomness of the secret directory.
@ -1541,29 +1532,29 @@ All options have been discussed separately in the sections above.
</property>
<!-- Transparent Encryption -->
<property>
<name>hbase.crypto.keyprovider</name>
<value>org.apache.hadoop.hbase.io.crypto.KeyStoreKeyProvider</value>
<name>hbase.crypto.keyprovider</name>
<value>org.apache.hadoop.hbase.io.crypto.KeyStoreKeyProvider</value>
</property>
<property>
<name>hbase.crypto.keyprovider.parameters</name>
<value>jceks:///path/to/hbase/conf/hbase.jks?password=***</value>
<name>hbase.crypto.keyprovider.parameters</name>
<value>jceks:///path/to/hbase/conf/hbase.jks?password=***</value>
</property>
<property>
<name>hbase.crypto.master.key.name</name>
<value>hbase</value>
<name>hbase.crypto.master.key.name</name>
<value>hbase</value>
</property>
<!-- WAL Encryption -->
<property>
<name>hbase.regionserver.hlog.reader.impl</name>
<value>org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader</value>
<name>hbase.regionserver.hlog.reader.impl</name>
<value>org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader</value>
</property>
<property>
<name>hbase.regionserver.hlog.writer.impl</name>
<value>org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter</value>
<name>hbase.regionserver.hlog.writer.impl</name>
<value>org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter</value>
</property>
<property>
<name>hbase.regionserver.wal.encryption</name>
<value>true</value>
<name>hbase.regionserver.wal.encryption</name>
<value>true</value>
</property>
<!-- For key rotation -->
<property>

View File

@ -38,13 +38,13 @@ To run the HBase shell, do as follows:
$ ./bin/hbase shell
----
Type +help+ and then +<RETURN>+ to see a listing of shell commands and options.
Browse at least the paragraphs at the end of the help emission for the gist of how variables and command arguments are entered into the HBase shell; in particular note how table names, rows, and columns, etc., must be quoted.
Type `help` and then `<RETURN>` to see a listing of shell commands and options.
Browse at least the paragraphs at the end of the help output for the gist of how variables and command arguments are entered into the HBase shell; in particular note how table names, rows, and columns, etc., must be quoted.
See <<shell_exercises,shell exercises>> for example basic shell operation.
See <<shell_exercises,shell exercises>> for example basic shell operation.
Here is a nicely formatted listing of link:http://learnhbase.wordpress.com/2013/03/02/hbase-shell-commands/[all shell
commands] by Rajeshbabu Chintaguntla.
commands] by Rajeshbabu Chintaguntla.
[[scripting]]
== Scripting with Ruby
@ -64,27 +64,26 @@ A new non-interactive mode has been added to the HBase Shell (link:https://issue
Non-interactive mode captures the exit status (success or failure) of HBase Shell commands and passes that status back to the command interpreter.
If you use the normal interactive mode, the HBase Shell will only ever return its own exit status, which will nearly always be `0` for success.
To invoke non-interactive mode, pass the +-n+ or +--non-interactive+ option to HBase Shell.
To invoke non-interactive mode, pass the `-n` or `--non-interactive` option to HBase Shell.
[[hbase.shell.noninteractive]]
== HBase Shell in OS Scripts
You can use the HBase shell from within operating system script interpreters like the Bash shell which is the default command interpreter for most Linux and UNIX distributions.
The following guidelines use Bash syntax, but could be adjusted to work with C-style shells such as csh or tcsh, and could probably be modified to work with the Microsoft Windows script interpreter as well.
Submissions are welcome.
The following guidelines use Bash syntax, but could be adjusted to work with C-style shells such as csh or tcsh, and could probably be modified to work with the Microsoft Windows script interpreter as well. Submissions are welcome.
NOTE: Spawning HBase Shell commands in this way is slow, so keep that in mind when you are deciding when combining HBase operations with the operating system command line is appropriate.
.Passing Commands to the HBase Shell
====
You can pass commands to the HBase Shell in non-interactive mode (see <<hbasee.shell.noninteractive,hbasee.shell.noninteractive>>) using the +echo+ command and the `|` (pipe) operator.
You can pass commands to the HBase Shell in non-interactive mode (see <<hbasee.shell.noninteractive,hbasee.shell.noninteractive>>) using the `echo` command and the `|` (pipe) operator.
Be sure to escape characters in the HBase commands which would otherwise be interpreted by the shell.
Some debug-level output has been truncated from the example below.
[source,bash]
----
$ echo "describe 'test1'" | ./hbase shell -n
Version 0.98.3-hadoop2, rd5e65a9144e315bb0a964e7730871af32f5018d5, Sat May 31 19:56:09 PDT 2014
describe 'test1'
@ -122,7 +121,7 @@ This is a naive script that shows one way to store the return value and make a d
echo "describe 'test'" | ./hbase shell -n > /dev/null 2>&1
status=$?
echo "The status was " $status
echo "The status was " $status
if ($status == 0); then
echo "The command succeeded"
else
@ -134,7 +133,7 @@ return $status
=== Checking for Success or Failure In Scripts
Getting an exit code of 0 means that the command you scripted definitely succeeded.
Getting an exit code of `0` means that the command you scripted definitely succeeded.
However, getting a non-zero exit code does not necessarily mean the command failed.
The command could have succeeded, but the client lost connectivity, or some other event obscured its success.
This is because RPC commands are stateless.
@ -163,10 +162,9 @@ enable 'test'
.Directing HBase Shell to Execute the Commands
====
Pass the path to the command file as the only argument to the +hbase
shell+ command.
Pass the path to the command file as the only argument to the `hbase shell` command.
Each command is executed and its output is shown.
If you do not include the +exit+ command in your script, you are returned to the HBase shell prompt.
If you do not include the `exit` command in your script, you are returned to the HBase shell prompt.
There is no way to programmatically check each individual command for success or failure.
Also, though you see the output for each command, the commands themselves are not echoed to the screen so it can be difficult to line up the command with its output.
@ -206,14 +204,14 @@ COLUMN CELL
== Passing VM Options to the Shell
You can pass VM options to the HBase Shell using the `HBASE_SHELL_OPTS` environment variable.
You can pass VM options to the HBase Shell using the `HBASE_SHELL_OPTS` environment variable.
You can set this in your environment, for instance by editing _~/.bashrc_, or set it as part of the command to launch HBase Shell.
The following example sets several garbage-collection-related variables, just for the lifetime of the VM running the HBase Shell.
The command should be run all on a single line, but is broken by the `\` character, for readability.
[source,bash]
----
$ HBASE_SHELL_OPTS="-verbose:gc -XX:+PrintGCApplicationStoppedTime -XX:+PrintGCDateStamps \
$ HBASE_SHELL_OPTS="-verbose:gc -XX:+PrintGCApplicationStoppedTime -XX:+PrintGCDateStamps \
-XX:+PrintGCDetails -Xloggc:$HBASE_HOME/logs/gc-hbase.log" ./bin/hbase shell
----
@ -221,10 +219,10 @@ $ HBASE_SHELL_OPTS="-verbose:gc -XX:+PrintGCApplicationStoppedTime -XX:+PrintGCD
=== Table variables
HBase 0.95 adds shell commands that provide a jruby-style object-oriented references for tables.
HBase 0.95 adds shell commands that provides jruby-style object-oriented references for tables.
Previously all of the shell commands that act upon a table have a procedural style that always took the name of the table as an argument.
HBase 0.95 introduces the ability to assign a table to a jruby variable.
The table reference can be used to perform data read write operations such as puts, scans, and gets well as admin functionality such as disabling, dropping, describing tables.
The table reference can be used to perform data read write operations such as puts, scans, and gets well as admin functionality such as disabling, dropping, describing tables.
For example, previously you would always specify a table name:
@ -234,17 +232,17 @@ hbase(main):000:0> create t, f
hbase(main):001:0> put 't', 'rold', 'f', 'v'
0 row(s) in 0.0080 seconds
hbase(main):002:0> scan 't'
ROW COLUMN+CELL
rold column=f:, timestamp=1378473207660, value=v
hbase(main):002:0> scan 't'
ROW COLUMN+CELL
rold column=f:, timestamp=1378473207660, value=v
1 row(s) in 0.0130 seconds
hbase(main):003:0> describe 't'
DESCRIPTION ENABLED
't', {NAME => 'f', DATA_BLOCK_ENCODING => 'NONE', BLOOMFILTER => 'ROW', REPLICATION_ true
SCOPE => '0', VERSIONS => '1', COMPRESSION => 'NONE', MIN_VERSIONS => '0', TTL => '2
147483647', KEEP_DELETED_CELLS => 'false', BLOCKSIZE => '65536', IN_MEMORY => 'false
', BLOCKCACHE => 'true'}
DESCRIPTION ENABLED
't', {NAME => 'f', DATA_BLOCK_ENCODING => 'NONE', BLOOMFILTER => 'ROW', REPLICATION_ true
SCOPE => '0', VERSIONS => '1', COMPRESSION => 'NONE', MIN_VERSIONS => '0', TTL => '2
147483647', KEEP_DELETED_CELLS => 'false', BLOCKSIZE => '65536', IN_MEMORY => 'false
', BLOCKCACHE => 'true'}
1 row(s) in 1.4430 seconds
hbase(main):004:0> disable 't'
@ -266,15 +264,15 @@ hbase(main):007 > t = create 't', 'f'
hbase(main):008 > t.put 'r', 'f', 'v'
0 row(s) in 0.0640 seconds
hbase(main):009 > t.scan
ROW COLUMN+CELL
r column=f:, timestamp=1331865816290, value=v
ROW COLUMN+CELL
r column=f:, timestamp=1331865816290, value=v
1 row(s) in 0.0110 seconds
hbase(main):010:0> t.describe
DESCRIPTION ENABLED
't', {NAME => 'f', DATA_BLOCK_ENCODING => 'NONE', BLOOMFILTER => 'ROW', REPLICATION_ true
SCOPE => '0', VERSIONS => '1', COMPRESSION => 'NONE', MIN_VERSIONS => '0', TTL => '2
147483647', KEEP_DELETED_CELLS => 'false', BLOCKSIZE => '65536', IN_MEMORY => 'false
', BLOCKCACHE => 'true'}
DESCRIPTION ENABLED
't', {NAME => 'f', DATA_BLOCK_ENCODING => 'NONE', BLOOMFILTER => 'ROW', REPLICATION_ true
SCOPE => '0', VERSIONS => '1', COMPRESSION => 'NONE', MIN_VERSIONS => '0', TTL => '2
147483647', KEEP_DELETED_CELLS => 'false', BLOCKSIZE => '65536', IN_MEMORY => 'false
', BLOCKCACHE => 'true'}
1 row(s) in 0.0210 seconds
hbase(main):038:0> t.disable
0 row(s) in 6.2350 seconds
@ -293,11 +291,11 @@ hbase(main):012:0> tab = get_table 't'
0 row(s) in 0.0010 seconds
=> Hbase::Table - t
hbase(main):013:0> tab.put r1 ,f, v
hbase(main):013:0> tab.put r1 ,f, v
0 row(s) in 0.0100 seconds
hbase(main):014:0> tab.scan
ROW COLUMN+CELL
r1 column=f:, timestamp=1378473876949, value=v
ROW COLUMN+CELL
r1 column=f:, timestamp=1378473876949, value=v
1 row(s) in 0.0240 seconds
hbase(main):015:0>
----
@ -308,8 +306,8 @@ The list_snapshots command also acts similarly.
----
hbase(main):016 > tables = list(t.*)
TABLE
t
TABLE
t
1 row(s) in 0.1040 seconds
=> #<#<Class:0x7677ce29>:0x21d377a4>
@ -333,7 +331,7 @@ IRB.conf[:SAVE_HISTORY] = 100
IRB.conf[:HISTORY_FILE] = "#{ENV['HOME']}/.irb-save-history"
----
See the +ruby+ documentation of _.irbrc_ to learn about other possible configurations.
See the `ruby` documentation of _.irbrc_ to learn about other possible configurations.
=== LOG data to timestamp
@ -352,7 +350,7 @@ hbase(main):021:0> import java.util.Date
hbase(main):022:0> Date.new(1218920189000).toString() => "Sat Aug 16 20:56:29 UTC 2008"
----
To output in a format that is exactly like that of the HBase log format will take a little messing with link:http://download.oracle.com/javase/6/docs/api/java/text/SimpleDateFormat.html[SimpleDateFormat].
To output in a format that is exactly like that of the HBase log format will take a little messing with link:http://download.oracle.com/javase/6/docs/api/java/text/SimpleDateFormat.html[SimpleDateFormat].
=== Debug
@ -368,7 +366,7 @@ hbase> debug <RETURN>
==== DEBUG log level
To enable DEBUG level logging in the shell, launch it with the +-d+ option.
To enable DEBUG level logging in the shell, launch it with the `-d` option.
[source,bash]
----
@ -380,13 +378,13 @@ $ ./bin/hbase shell -d
==== count
Count command returns the number of rows in a table.
It's quite fast when configured with the right CACHE
It's quite fast when configured with the right CACHE
[source]
----
hbase> count '<tablename>', CACHE => 1000
----
----
The above count fetches 1000 rows at a time.
Set CACHE lower if your rows are big.
Default is to fetch one row at a time.
Default is to fetch one row at a time.

View File

@ -31,18 +31,18 @@
Apache link:http://thrift.apache.org/[Thrift] is a cross-platform, cross-language development framework.
HBase includes a Thrift API and filter language.
The Thrift API relies on client and server processes.
Documentation about the HBase Thrift API is located at link:http://wiki.apache.org/hadoop/Hbase/ThriftApi.
Documentation about the HBase Thrift API is located at http://wiki.apache.org/hadoop/Hbase/ThriftApi.
You can configure Thrift for secure authentication at the server and client side, by following the procedures in <<security.client.thrift,security.client.thrift>> and <<security.gateway.thrift,security.gateway.thrift>>.
You can configure Thrift for secure authentication at the server and client side, by following the procedures in <<security.client.thrift>> and <<security.gateway.thrift>>.
The rest of this chapter discusses the filter language provided by the Thrift API.
[[thrift.filter_language]]
== Filter Language
Thrift Filter Language was introduced in APache HBase 0.92.
Thrift Filter Language was introduced in HBase 0.92.
It allows you to perform server-side filtering when accessing HBase over Thrift or in the HBase shell.
You can find out more about shell integration by using the `scan help` command in the shell.
You can find out more about shell integration by using the `scan help` command in the shell.
You specify a filter as a string, which is parsed on the server to construct the filter.
@ -69,7 +69,7 @@ Keep the following syntax guidelines in mind.
.Binary Operators
`AND`::
If the `AND` operator is used, the key-vallue must satisfy both the filters.
If the `AND` operator is used, the key-value must satisfy both filters.
`OR`::
If the `OR` operator is used, the key-value must satisfy at least one of the filters.
@ -79,7 +79,7 @@ Keep the following syntax guidelines in mind.
For a particular row, if any of the key-values fail the filter condition, the entire row is skipped.
`WHILE`::
For a particular row, key-values will be emitted until a key-value is reached t hat fails the filter condition.
For a particular row, key-values will be emitted until a key-value is reached that fails the filter condition.
.Compound Operators
====
@ -142,8 +142,7 @@ A comparator can be any of the following:
The comparison is case insensitive.
Only EQUAL and NOT_EQUAL comparisons are valid with this comparator
The general syntax of a comparator is:`
ComparatorType:ComparatorValue`
The general syntax of a comparator is: `ComparatorType:ComparatorValue`
The ComparatorType for the various comparators is as follows:
@ -155,7 +154,7 @@ The ComparatorType for the various comparators is as follows:
The ComparatorValue can be any value.
.Example ComparatorValues
. `binary:abc` will match everything that is lexicographically greater than "abc"
. `binary:abc` will match everything that is lexicographically greater than "abc"
. `binaryprefix:abc` will match everything whose first 3 characters are lexicographically equal to "abc"
. `regexstring:ab*yz` will match everything that doesn't begin with "ab" and ends with "yz"
. `substring:abc123` will match everything that begins with the substring "abc123"
@ -165,48 +164,39 @@ The ComparatorValue can be any value.
[source,php]
----
<? $_SERVER['PHP_ROOT'] = realpath(dirname(__FILE__).'/..');
require_once $_SERVER['PHP_ROOT'].'/flib/__flib.php';
flib_init(FLIB_CONTEXT_SCRIPT);
require_module('storage/hbase');
$hbase = new HBase('<server_name_running_thrift_server>', <port on which thrift server is running>);
$hbase->open();
$client = $hbase->getClient();
$result = $client->scannerOpenWithFilterString('table_name', "(PrefixFilter ('row2') AND (QualifierFilter (>=, 'binary:xyz'))) AND (TimestampsFilter ( 123, 456))");
$to_print = $client->scannerGetList($result,1);
while ($to_print) {
print_r($to_print);
$to_print = $client->scannerGetList($result,1);
}
$client->scannerClose($result);
<?
$_SERVER['PHP_ROOT'] = realpath(dirname(__FILE__).'/..');
require_once $_SERVER['PHP_ROOT'].'/flib/__flib.php';
flib_init(FLIB_CONTEXT_SCRIPT);
require_module('storage/hbase');
$hbase = new HBase('<server_name_running_thrift_server>', <port on which thrift server is running>);
$hbase->open();
$client = $hbase->getClient();
$result = $client->scannerOpenWithFilterString('table_name', "(PrefixFilter ('row2') AND (QualifierFilter (>=, 'binary:xyz'))) AND (TimestampsFilter ( 123, 456))");
$to_print = $client->scannerGetList($result,1);
while ($to_print) {
print_r($to_print);
$to_print = $client->scannerGetList($result,1);
}
$client->scannerClose($result);
?>
----
=== Example Filter Strings
* `“PrefixFilter (Row) AND PageFilter (1) AND FirstKeyOnlyFilter
()”` will return all key-value pairs that match the following conditions:
* `"PrefixFilter ('Row') AND PageFilter (1) AND FirstKeyOnlyFilter ()"` will return all key-value pairs that match the following conditions:
+
. The row containing the key-value should have prefix ``Row''
. The key-value must be located in the first row of the table
. The key-value pair must be the first key-value in the row
* `“(RowFilter (=, binary:Row 1) AND TimeStampsFilter (74689,
89734)) OR ColumnRangeFilter (abc, true, xyz,
false))”` will return all key-value pairs that match both the following conditions:
. The row containing the key-value should have prefix _Row_
. The key-value must be located in the first row of the table
. The key-value pair must be the first key-value in the row
+
* The key-value is in a row having row key ``Row 1''
* The key-value must have a timestamp of either 74689 or 89734.
* Or it must match the following condition:
* `"(RowFilter (=, 'binary:Row 1') AND TimeStampsFilter (74689, 89734)) OR ColumnRangeFilter ('abc', true, 'xyz', false))"` will return all key-value pairs that match both the following conditions:
** The key-value is in a row having row key _Row 1_
** The key-value must have a timestamp of either 74689 or 89734.
** Or it must match the following condition:
*** The key-value pair must be in a column that is lexicographically >= abc and < xyz 
+
* The key-value pair must be in a column that is lexicographically >= abc and < xyz 
* `“SKIP ValueFilter (0)”` will skip the entire row if any of the values in the row is not 0
* `"SKIP ValueFilter (0)"` will skip the entire row if any of the values in the row is not 0
[[individualfiltersyntax]]
=== Individual Filter Syntax
@ -279,7 +269,7 @@ SingleColumnValueFilter::
This filter takes a column family, a qualifier, a compare operator and a comparator.
If the specified column is not found all the columns of that row will be emitted.
If the column is found and the comparison with the comparator returns true, all the columns of the row will be emitted.
If the condition fails, the row will not be emitted.
If the condition fails, the row will not be emitted.
SingleColumnValueExcludeFilter::
This filter takes the same arguments and behaves same as SingleColumnValueFilter however, if the column is found and the condition passes, all the columns of the row will be emitted except for the tested column value.

File diff suppressed because it is too large Load Diff

View File

@ -27,9 +27,9 @@
:icons: font
:experimental:
You cannot skip major versions upgrading. If you are upgrading from version 0.90.x to 0.94.x, you must first go from 0.90.x to 0.92.x and then go from 0.92.x to 0.94.x.
You cannot skip major versions when upgrading. If you are upgrading from version 0.90.x to 0.94.x, you must first go from 0.90.x to 0.92.x and then go from 0.92.x to 0.94.x.
Note:It may be possible to skip across versions -- for example go from 0.92.2 straight to 0.98.0 just following the 0.96.x upgrade instructions -- but we have not tried it so cannot say whether it works or not.
NOTE: It may be possible to skip across versions -- for example go from 0.92.2 straight to 0.98.0 just following the 0.96.x upgrade instructions -- but these scenarios are untested.
Review <<configuration>>, in particular <<hadoop>>.
@ -41,7 +41,7 @@ HBase has two versioning schemes, pre-1.0 and post-1.0. Both are detailed below.
[[hbase.versioning.post10]]
=== Post 1.0 versions
Starting with 1.0.0 release, HBase uses link:http://semver.org/[Semantic Versioning] for its release versioning. In summary:
Starting with the 1.0.0 release, HBase uses link:http://semver.org/[Semantic Versioning] for its release versioning. In summary:
.Given a version number MAJOR.MINOR.PATCH, increment the:
* MAJOR version when you make incompatible API changes,
@ -90,7 +90,7 @@ In addition to the usual API versioning considerations HBase has other compatibi
.Operational Compatibility
* Metric changes
* Behavioral changes of services
*Web page APIs
* Web page APIs
.Summary
* A patch upgrade is a drop-in replacement. Any change that is not Java binary compatible would not be allowed.footnote:[See http://docs.oracle.com/javase/specs/jls/se7/html/jls-13.html.]
@ -149,25 +149,25 @@ Our first "Development" Series was the 0.89 set that came out ahead of HBase 0.9
[[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 point versions: e.g. from 0.94.5 to 0.94.6. See link:[Does compatibility between versions also mean binary compatibility?] discussion on the hbaes 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 point versions: e.g. from 0.94.5 to 0.94.6. See link:[Does compatibility between versions also mean binary compatibility?] discussion on the HBase dev mailing list.
[[hbase.rolling.upgrade]]
=== Rolling Upgrades
A rolling upgrade is the process by which you update the servers in your cluster a server at a time. You can rolling upgrade across HBase versions if they are binary or wire compatible. See <<hbase.rolling.restart>> for more on what this means. Coarsely, a rolling upgrade is a graceful stop each server, update the software, and then restart. You do this for each server in the cluster. Usually you upgrade the Master first and then the regionservers. See <<rolling>> for tools that can help use the rolling upgrade process.
A rolling upgrade is the process by which you update the servers in your cluster a server at a time. You can rolling upgrade across HBase versions if they are binary or wire compatible. See <<hbase.rolling.restart>> for more on what this means. Coarsely, a rolling upgrade is a graceful stop each server, update the software, and then restart. You do this for each server in the cluster. Usually you upgrade the Master first and then the RegionServers. See <<rolling>> for tools that can help use the rolling upgrade process.
For example, in the below, hbase was symlinked to the actual hbase install. On upgrade, before running a rolling restart over the cluser, we changed the symlink to point at the new HBase software version and then ran
For example, in the below, HBase was symlinked to the actual HBase install. On upgrade, before running a rolling restart over the cluser, we changed the symlink to point at the new HBase software version and then ran
[source,bash]
----
$ HADOOP_HOME=~/hadoop-2.6.0-CRC-SNAPSHOT ~/hbase/bin/rolling-restart.sh --config ~/conf_hbase
----
The rolling-restart script will first gracefully stop and restart the master, and then each of the regionservers in turn. Because the symlink was changed, on restart the server will come up using the new hbase version. Check logs for errors as the rolling upgrade proceeds.
The rolling-restart script will first gracefully stop and restart the master, and then each of the RegionServers in turn. Because the symlink was changed, on restart the server will come up using the new HBase version. Check logs for errors as the rolling upgrade proceeds.
[[hbase.rolling.restart]]
.Rolling Upgrade Between Versions that are Binary/Wire Compatible
Unless otherwise specified, HBase point versions are binary compatible. You can do a <<hbase.rolling.upgrade>> between hbase point versions. For example, you can go to 0.94.6 from 0.94.5 by doing a rolling upgrade across the cluster replacing the 0.94.5 binary with a 0.94.6 binary.
Unless otherwise specified, HBase point versions are binary compatible. You can do a <<hbase.rolling.upgrade>> between HBase point versions. For example, you can go to 0.94.6 from 0.94.5 by doing a rolling upgrade across the cluster replacing the 0.94.5 binary with a 0.94.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>>. For example, in <<upgrade1.0.rolling.upgrade>>, we state that it is possible to do a rolling upgrade between hbase-0.98.x and hbase-1.0.0.
@ -176,7 +176,7 @@ In the minor version-particular sections below, we call out where the versions a
[[upgrade1.0]]
=== Upgrading from 0.98.x to 1.0.x
In this section we first note the significant changes that come in with 1.0.0 HBase and then we go over the upgrade process. Be sure to read the significant changes section with care so you avoid surprises.
In this section we first note the significant changes that come in with 1.0.0 HBase and then we go over the upgrade process. Be sure to read the significant changes section with care so you avoid surprises.
==== Changes of Note!
@ -188,30 +188,30 @@ See <<zookeeper.requirements>>.
[[default.ports.changed]]
.HBase Default Ports Changed
The ports used by HBase changed. The used to be in the 600XX range. In hbase-1.0.0 they have been moved up out of the ephemeral port range and are 160XX instead (Master web UI was 60010 and is now 16010; the RegionServer web UI was 60030 and is now 16030, etc). If you want to keep the old port locations, copy the port setting configs from _hbase-default.xml_ into _hbase-site.xml_, change them back to the old values from hbase-0.98.x era, and ensure you've distributed your configurations before you restart.
The ports used by HBase changed. They used to be in the 600XX range. In HBase 1.0.0 they have been moved up out of the ephemeral port range and are 160XX instead (Master web UI was 60010 and is now 16010; the RegionServer web UI was 60030 and is now 16030, etc.). If you want to keep the old port locations, copy the port setting configs from _hbase-default.xml_ into _hbase-site.xml_, change them back to the old values from the HBase 0.98.x era, and ensure you've distributed your configurations before you restart.
[[upgrade1.0.hbase.bucketcache.percentage.in.combinedcache]]
.hbase.bucketcache.percentage.in.combinedcache configuration has been REMOVED
You may have made use of this configuration if you are using BucketCache. If NOT using BucketCache, this change does not effect you. Its removal means that your L1 LruBlockCache is now sized using `hfile.block.cache.size` -- i.e. the way you would size the onheap L1 LruBlockCache if you were NOT doing BucketCache -- and the BucketCache size is not whatever the setting for hbase.bucketcache.size is. You may need to adjust configs to get the LruBlockCache and BucketCache sizes set to what they were in 0.98.x and previous. If you did not set this config., its default value was 0.9. If you do nothing, your BucketCache will increase in size by 10%. Your L1 LruBlockCache will become hfile.block.cache.size times your java heap size (`hfile.block.cache.size` is a float between 0.0 and 1.0). To read more, see link:https://issues.apache.org/jira/browse/HBASE-11520[HBASE-11520 Simplify offheap cache config by removing the confusing "hbase.bucketcache.percentage.in.combinedcache"].
You may have made use of this configuration if you are using BucketCache. If NOT using BucketCache, this change does not effect you. Its removal means that your L1 LruBlockCache is now sized using `hfile.block.cache.size` -- i.e. the way you would size the on-heap L1 LruBlockCache if you were NOT doing BucketCache -- and the BucketCache size is not whatever the setting for `hbase.bucketcache.size` is. You may need to adjust configs to get the LruBlockCache and BucketCache sizes set to what they were in 0.98.x and previous. If you did not set this config., its default value was 0.9. If you do nothing, your BucketCache will increase in size by 10%. Your L1 LruBlockCache will become `hfile.block.cache.size` times your java heap size (`hfile.block.cache.size` is a float between 0.0 and 1.0). To read more, see link:https://issues.apache.org/jira/browse/HBASE-11520[HBASE-11520 Simplify offheap cache config by removing the confusing "hbase.bucketcache.percentage.in.combinedcache"].
[[hbase-12068]]
.If you have your own customer filters....
.If you have your own customer filters.
See the release notes on the issue link:https://issues.apache.org/jira/browse/HBASE-12068[HBASE-12068 [Branch-1\] Avoid need to always do KeyValueUtil#ensureKeyValue for Filter transformCell]; be sure to follow the recommendations therein.
[[dlr]]
.Distributed Log Replay
<<distributed.log.replay>> is off by default in hbase-1.0. Enabling it can make a big difference improving HBase MTTR. Enable this feature if you are doing a clean stop/start when you are upgrading. You cannot rolling upgrade on to this feature (caveat if you are running on a version of hbase in excess of hbase-0.98.4 -- see link:https://issues.apache.org/jira/browse/HBASE-12577[HBASE-12577 Disable distributed log replay by default] for more).
<<distributed.log.replay>> is off by default in HBase 1.0.0. Enabling it can make a big difference improving HBase MTTR. Enable this feature if you are doing a clean stop/start when you are upgrading. You cannot rolling upgrade to this feature (caveat if you are running on a version of HBase in excess of HBase 0.98.4 -- see link:https://issues.apache.org/jira/browse/HBASE-12577[HBASE-12577 Disable distributed log replay by default] for more).
[[upgrade1.0.rolling.upgrade]]
==== Rolling upgrade from 0.98.x to HBase 1.0.0
.From 0.96.x to 1.0.0
NOTE: You cannot do a <<hbase.rolling.upgrade,rolling upgrade>> from 0.96.x to 1.0.0 without first doing a rolling upgrade to 0.98.x. See comment in link:https://issues.apache.org/jira/browse/HBASE-11164?focusedCommentId=14182330&amp;page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&#35;comment-14182330[HBASE-11164 Document and test rolling updates from 0.98 -> 1.0] for the why. Also because hbase-1.0.0 enables hfilev3 by default, link:https://issues.apache.org/jira/browse/HBASE-9801[HBASE-9801 Change the default HFile version to V3], and support for hfilev3 only arrives in 0.98, this is another reason you cannot rolling upgrade from hbase-0.96.x; if the rolling upgrade stalls, the 0.96.x servers cannot open files written by the servers running the newer hbase-1.0.0 hfilev3 writing servers.
NOTE: You cannot do a <<hbase.rolling.upgrade,rolling upgrade>> from 0.96.x to 1.0.0 without first doing a rolling upgrade to 0.98.x. See comment in link:https://issues.apache.org/jira/browse/HBASE-11164?focusedCommentId=14182330&amp;page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&#35;comment-14182330[HBASE-11164 Document and test rolling updates from 0.98 -> 1.0] for the why. Also because HBase 1.0.0 enables HFile v3 by default, link:https://issues.apache.org/jira/browse/HBASE-9801[HBASE-9801 Change the default HFile version to V3], and support for HFile v3 only arrives in 0.98, this is another reason you cannot rolling upgrade from HBase 0.96.x; if the rolling upgrade stalls, the 0.96.x servers cannot open files written by the servers running the newer HBase 1.0.0 with HFile's of version 3.
There are no known issues running a <<hbase.rolling.upgrade,rolling upgrade>> from hbase-0.98.x to hbase-1.0.0.
There are no known issues running a <<hbase.rolling.upgrade,rolling upgrade>> from HBase 0.98.x to HBase 1.0.0.
[[upgrade1.0.from.0.94]]
==== Upgrading to 1.0 from 0.94
You cannot rolling upgrade from 0.94.x to 1.x.x. You must stop your cluster, install the 1.x.x software, run the migration described at <<executing.the.0.96.upgrade>> (substituting 1.x.x. wherever we make mention of 0.96.x in the section below), and then restart. Be sure to upgrade your zookeeper if it is a version less than the required 3.4.x.
You cannot rolling upgrade from 0.94.x to 1.x.x. You must stop your cluster, install the 1.x.x software, run the migration described at <<executing.the.0.96.upgrade>> (substituting 1.x.x. wherever we make mention of 0.96.x in the section below), and then restart. Be sure to upgrade your ZooKeeper if it is a version less than the required 3.4.x.
[[upgrade0.98]]
=== Upgrading from 0.96.x to 0.98.x
@ -230,7 +230,7 @@ A rolling upgrade from 0.94.x directly to 0.98.x does not work. The upgrade path
==== The "Singularity"
.HBase 0.96.x was EOL'd, September 1st, 2014
NOTE: Do not deploy 0.96.x Deploy a 0.98.x at least. See link:https://issues.apache.org/jira/browse/HBASE-11642[EOL 0.96].
NOTE: Do not deploy 0.96.x Deploy at least 0.98.x. See link:https://issues.apache.org/jira/browse/HBASE-11642[EOL 0.96].
You will have to stop your old 0.94.x cluster completely to upgrade. If you are replicating between clusters, both clusters will have to go down to upgrade. Make sure it is a clean shutdown. The less WAL files around, the faster the upgrade will run (the upgrade will split any log files it finds in the filesystem as part of the upgrade process). All clients must be upgraded to 0.96 too.
@ -242,7 +242,7 @@ The API has changed. You will need to recompile your code against 0.96 and you m
.HDFS and ZooKeeper must be up!
NOTE: HDFS and ZooKeeper should be up and running during the upgrade process.
hbase-0.96.0 comes with an upgrade script. Run
HBase 0.96.0 comes with an upgrade script. Run
[source,bash]
----
@ -251,11 +251,11 @@ $ bin/hbase upgrade
to see its usage. The script has two main modes: `-check`, and `-execute`.
.check
The check step is run against a running 0.94 cluster. Run it from a downloaded 0.96.x binary. The check step is looking for the presence of HFileV1 files. These are unsupported in hbase-0.96.0. To purge them -- have them rewritten as HFileV2 -- you must run a compaction.
The check step is run against a running 0.94 cluster. Run it from a downloaded 0.96.x binary. The check step is looking for the presence of HFile v1 files. These are unsupported in HBase 0.96.0. To have them rewritten as HFile v2 you must run a compaction.
The check step prints stats at the end of its run (grep for `“Result:”` in the log) printing absolute path of the tables it scanned, any HFileV1 files found, the regions containing said files (the regions we need to major compact to purge the HFileV1s), and any corrupted files if any found. A corrupt file is unreadable, and so is undefined (neither HFileV1 nor HFileV2).
The check step prints stats at the end of its run (grep for `“Result:”` in the log) printing absolute path of the tables it scanned, any HFile v1 files found, the regions containing said files (these regions will need a major compaction), and any corrupted files if found. A corrupt file is unreadable, and so is undefined (neither HFile v1 nor HFile v2).
To run the check step, run
To run the check step, run
[source,bash]
----
@ -286,23 +286,23 @@ hdfs://localhost:41020/myHBase/usertable/ecdd3eaee2d2fcf8184ac025555bb2af
There are some HFileV1, or corrupt files (files with incorrect major version)
----
In the above sample output, there are two HFileV1 in two regions, and one corrupt file. Corrupt files should probably be removed. The regions that have HFileV1s need to be major compacted. To major compact, start up the hbase shell and review how to compact an individual region. After the major compaction is done, rerun the check step and the HFileV1s shoudl be gone, replaced by HFileV2 instances.
In the above sample output, there are two HFile v1 files in two regions, and one corrupt file. Corrupt files should probably be removed. The regions that have HFile v1s need to be major compacted. To major compact, start up the hbase shell and review how to compact an individual region. After the major compaction is done, rerun the check step and the HFile v1 files should be gone, replaced by HFile v2 instances.
By default, the check step scans the hbase root directory (defined as hbase.rootdir in the configuration). To scan a specific directory only, pass the -dir option.
By default, the check step scans the HBase root directory (defined as `hbase.rootdir` in the configuration). To scan a specific directory only, pass the `-dir` option.
[source,bash]
----
$ bin/hbase upgrade -check -dir /myHBase/testTable
----
The above command would detect HFileV1s in the /myHBase/testTable directory.
The above command would detect HFile v1 files in the _/myHBase/testTable_ directory.
Once the check step reports all the HFileV1 files have been rewritten, it is safe to proceed with the upgrade.
Once the check step reports all the HFile v1 files have been rewritten, it is safe to proceed with the upgrade.
.execute
After the _check_ step shows the cluster is free of HFileV1, it is safe to proceed with the upgrade. Next is the _execute_ step. You must *SHUTDOWN YOUR 0.94.x CLUSTER* before you can run the execute step. The execute step will not run if it detects running HBase masters or regionservers.
After the _check_ step shows the cluster is free of HFile v1, it is safe to proceed with the upgrade. Next is the _execute_ step. You must *SHUTDOWN YOUR 0.94.x CLUSTER* before you can run the execute step. The execute step will not run if it detects running HBase masters or RegionServers.
[NOTE]
====
HDFS and ZooKeeper should be up and running during the upgrade process. If zookeeper is managed by HBase, then you can start zookeeper so it is available to the upgrade by running
HDFS and ZooKeeper should be up and running during the upgrade process. If zookeeper is managed by HBase, then you can start zookeeper so it is available to the upgrade by running
[source,bash]
----
$ ./hbase/bin/hbase-daemon.sh start zookeeper
@ -317,7 +317,7 @@ The execute upgrade step is made of three substeps.
* WAL Log Splitting: If the 0.94.x cluster shutdown was not clean, we'll split WAL logs as part of migration before we startup on 0.96.0. This WAL splitting runs slower than the native distributed WAL splitting because it is all inside the single upgrade process (so try and get a clean shutdown of the 0.94.0 cluster if you can).
To run the _execute_ step, make sure that first you have copied hbase-0.96.0 binaries everywhere under servers and under clients. Make sure the 0.94.0 cluster is down. Then do as follows:
To run the _execute_ step, make sure that first you have copied HBase 0.96.0 binaries everywhere under servers and under clients. Make sure the 0.94.0 cluster is down. Then do as follows:
[source,bash]
----
$ bin/hbase upgrade -execute
@ -339,7 +339,7 @@ Starting Log splitting
...
Successfully completed Log splitting
----
If the output from the execute step looks good, stop the zookeeper instance you started to do the upgrade:
[source,bash]
----
@ -376,22 +376,22 @@ The migration is a one-time event. However, every time your cluster starts, `MET
[[upgrade0.94]]
=== Upgrading from 0.92.x to 0.94.x
We used to think that 0.92 and 0.94 were interface compatible and that you can do a rolling upgrade between these versions but then we figured that link:https://issues.apache.org/jira/browse/HBASE-5357[">]HBASE-5357 Use builder pattern in HColumnDescriptor] changed method signatures so rather than return void they instead return HColumnDescriptor. This will throw`java.lang.NoSuchMethodError: org.apache.hadoop.hbase.HColumnDescriptor.setMaxVersions(I)V` so 0.92 and 0.94 are NOT compatible. You cannot do a rolling upgrade between them.
We used to think that 0.92 and 0.94 were interface compatible and that you can do a rolling upgrade between these versions but then we figured that link:https://issues.apache.org/jira/browse/HBASE-5357[HBASE-5357 Use builder pattern in HColumnDescriptor] changed method signatures so rather than return `void` they instead return `HColumnDescriptor`. This will throw`java.lang.NoSuchMethodError: org.apache.hadoop.hbase.HColumnDescriptor.setMaxVersions(I)V` so 0.92 and 0.94 are NOT compatible. You cannot do a rolling upgrade between them.
[[upgrade0.92]]
=== Upgrading from 0.90.x to 0.92.x
==== Upgrade Guide
ou will find that 0.92.0 runs a little differently to 0.90.x releases. Here are a few things to watch out for upgrading from 0.90.x to 0.92.0.
You will find that 0.92.0 runs a little differently to 0.90.x releases. Here are a few things to watch out for upgrading from 0.90.x to 0.92.0.
.tl:dr
[NOTE]
====
If you've not patience, here are the important things to know upgrading.
These are the important things to know before upgrading.
. Once you upgrade, you cant go back.
. MSLAB is on by default. Watch that heap usage if you have a lot of regions.
. Distributed Log Splitting is on by default. It should make region server failover faster.
. Distributed Log Splitting is on by default. It should make RegionServer failover faster.
. Theres a separate tarball for security.
@ -399,10 +399,10 @@ If you've not patience, here are the important things to know upgrading.
====
.You cant go back!
To move to 0.92.0, all you need to do is shutdown your cluster, replace your hbase 0.90.x with hbase 0.92.0 binaries (be sure you clear out all 0.90.x instances) and restart (You cannot do a rolling restart from 0.90.x to 0.92.x -- you must restart). On startup, the `.META.` table content is rewritten removing the table schema from the `info:regioninfo` column. Also, any flushes done post first startup will write out data in the new 0.92.0 file format, <<hfilev2>>. This means you cannot go back to 0.90.x once youve started HBase 0.92.0 over your HBase data directory.
To move to 0.92.0, all you need to do is shutdown your cluster, replace your HBase 0.90.x with HBase 0.92.0 binaries (be sure you clear out all 0.90.x instances) and restart (You cannot do a rolling restart from 0.90.x to 0.92.x -- you must restart). On startup, the `.META.` table content is rewritten removing the table schema from the `info:regioninfo` column. Also, any flushes done post first startup will write out data in the new 0.92.0 file format, <<hfilev2>>. This means you cannot go back to 0.90.x once youve started HBase 0.92.0 over your HBase data directory.
.MSLAB is ON by default
In 0.92.0, the `<<hbase.hregion.memstore.mslab.enabled,hbase.hregion.memstore.mslab.enabled>>` flag is set to `true` (See <<gcpause>>). In 0.90.x it was false. When it is enabled, memstores will step allocate memory in MSLAB 2MB chunks even if the memstore has zero or just a few small elements. This is fine usually but if you had lots of regions per regionserver in a 0.90.x cluster (and MSLAB was off), you may find yourself OOME'ing on upgrade because the `thousands of regions * number of column families * 2MB MSLAB` (at a minimum) puts your heap over the top. Set `hbase.hregion.memstore.mslab.enabled` to `false` or set the MSLAB size down from 2MB by setting `hbase.hregion.memstore.mslab.chunksize` to something less.
In 0.92.0, the `<<hbase.hregion.memstore.mslab.enabled,hbase.hregion.memstore.mslab.enabled>>` flag is set to `true` (See <<gcpause>>). In 0.90.x it was false. When it is enabled, memstores will step allocate memory in MSLAB 2MB chunks even if the memstore has zero or just a few small elements. This is fine usually but if you had lots of regions per RegionServer in a 0.90.x cluster (and MSLAB was off), you may find yourself OOME'ing on upgrade because the `thousands of regions * number of column families * 2MB MSLAB` (at a minimum) puts your heap over the top. Set `hbase.hregion.memstore.mslab.enabled` to `false` or set the MSLAB size down from 2MB by setting `hbase.hregion.memstore.mslab.chunksize` to something less.
[[dls]]
.Distributed Log Splitting is on by default
@ -412,18 +412,18 @@ Previous, WAL logs on crash were split by the Master alone. In 0.92.0, log split
In 0.92.0, <<hfilev2>> indices and bloom filters take up residence in the same LRU used caching blocks that come from the filesystem. In 0.90.x, the HFile v1 indices lived outside of the LRU so they took up space even if the index was on a cold file, one that wasnt being actively used. With the indices now in the LRU, you may find you have less space for block caching. Adjust your block cache accordingly. See the <<block.cache>> for more detail. The block size default size has been changed in 0.92.0 from 0.2 (20 percent of heap) to 0.25.
.On the Hadoop version to use
Run 0.92.0 on Hadoop 1.0.x (or CDH3u3 when it ships). The performance benefits are worth making the move. Otherwise, our Hadoop prescription is as it has been; you need an Hadoop that supports a working sync. See <<hadoop>>.
Run 0.92.0 on Hadoop 1.0.x (or CDH3u3). The performance benefits are worth making the move. Otherwise, our Hadoop prescription is as it has been; you need an Hadoop that supports a working sync. See <<hadoop>>.
If running on Hadoop 1.0.x (or CDH3u3), enable local read. See link:http://files.meetup.com/1350427/hug_ebay_jdcryans.pdf[Practical Caching] presentation for ruminations on the performance benefits going local (and for how to enable local reads).
.HBase 0.92.0 ships with ZooKeeper 3.4.2
If you can, upgrade your zookeeper. If you cant, 3.4.2 clients should work against 3.3.X ensembles (HBase makes use of 3.4.2 API).
If you can, upgrade your ZooKeeper. If you cant, 3.4.2 clients should work against 3.3.X ensembles (HBase makes use of 3.4.2 API).
.Online alter is off by default
In 0.92.0, weve added an experimental online schema alter facility (See <<hbase.online.schema.update.enable,hbase.online.schema.update.enable>>). Its off by default. Enable it at your own risk. Online alter and splitting tables do not play well together so be sure your cluster quiescent using this feature (for now).
In 0.92.0, weve added an experimental online schema alter facility (See <<hbase.online.schema.update.enable,hbase.online.schema.update.enable>>). It's off by default. Enable it at your own risk. Online alter and splitting tables do not play well together so be sure your cluster quiescent using this feature (for now).
.WebUI
The webui has had a few additions made in 0.92.0. It now shows a list of the regions currently transitioning, recent compactions/flushes, and a process list of running processes (usually empty if all is well and requests are being handled promptly). Other additions including requests by region, a debugging servlet dump, etc.
The web UI has had a few additions made in 0.92.0. It now shows a list of the regions currently transitioning, recent compactions/flushes, and a process list of running processes (usually empty if all is well and requests are being handled promptly). Other additions including requests by region, a debugging servlet dump, etc.
.Security tarball
We now ship with two tarballs; secure and insecure HBase. Documentation on how to setup a secure HBase is on the way.
@ -432,10 +432,10 @@ We now ship with two tarballs; secure and insecure HBase. Documentation on how t
0.92.0 adds two new features: multi-slave and multi-master replication. The way to enable this is the same as adding a new peer, so in order to have multi-master you would just run add_peer for each cluster that acts as a master to the other slave clusters. Collisions are handled at the timestamp level which may or may not be what you want, this needs to be evaluated on a per use case basis. Replication is still experimental in 0.92 and is disabled by default, run it at your own risk.
.RegionServer now aborts if OOME
If an OOME, we now have the JVM kill -9 the regionserver process so it goes down fast. Previous, a RegionServer might stick around after incurring an OOME limping along in some wounded state. To disable this facility, and recommend you leave it in place, youd need to edit the bin/hbase file. Look for the addition of the -XX:OnOutOfMemoryError="kill -9 %p" arguments (See link:https://issues.apache.org/jira/browse/HBASE-4769[HBASE-4769 - Abort RegionServer Immediately on OOME]).
If an OOME, we now have the JVM kill -9 the RegionServer process so it goes down fast. Previous, a RegionServer might stick around after incurring an OOME limping along in some wounded state. To disable this facility, and recommend you leave it in place, youd need to edit the bin/hbase file. Look for the addition of the -XX:OnOutOfMemoryError="kill -9 %p" arguments (See link:https://issues.apache.org/jira/browse/HBASE-4769[HBASE-4769 - Abort RegionServer Immediately on OOME]).
.HFile V2 and the “Bigger, Fewer” Tendency
0.92.0 stores data in a new format, <<hfilev2>>. As HBase runs, it will move all your data from HFile v1 to HFile v2 format. This auto-migration will run in the background as flushes and compactions run. HFile V2 allows HBase run with larger regions/files. In fact, we encourage that all HBasers going forward tend toward Facebook axiom #1, run with larger, fewer regions. If you have lots of regions now -- more than 100s per host -- you should look into setting your region size up after you move to 0.92.0 (In 0.92.0, default size is now 1G, up from 256M), and then running online merge tool (See link:https://issues.apache.org/jira/browse/HBASE-1621[HBASE-1621 merge tool should work on online cluster, but disabled table]).
.HFile v2 and the “Bigger, Fewer” Tendency
0.92.0 stores data in a new format, <<hfilev2>>. As HBase runs, it will move all your data from HFile v1 to HFile v2 format. This auto-migration will run in the background as flushes and compactions run. HFile v2 allows HBase run with larger regions/files. In fact, we encourage that all HBasers going forward tend toward Facebook axiom #1, run with larger, fewer regions. If you have lots of regions now -- more than 100s per host -- you should look into setting your region size up after you move to 0.92.0 (In 0.92.0, default size is now 1G, up from 256M), and then running online merge tool (See link:https://issues.apache.org/jira/browse/HBASE-1621[HBASE-1621 merge tool should work on online cluster, but disabled table]).
[[upgrade0.90]]
=== Upgrading to HBase 0.90.x from 0.20.x or 0.89.x
@ -447,4 +447,4 @@ Finally, if upgrading from 0.20.x, check your .META. schema in the shell. In the
----
hbase> scan '-ROOT-'
----
in the shell. This will output the current `.META.` schema. Check `MEMSTORE_FLUSHSIZE` size. Is it 16kb (16384)? If so, you will need to change this (The 'normal'/default value is 64MB (67108864)). Run the script `bin/set_meta_memstore_size.rb`. This will make the necessary edit to your `.META.` schema. Failure to run this change will make for a slow cluster. See link:https://issues.apache.org/jira/browse/HBASE-3499[HBASE-3499 Users upgrading to 0.90.0 need to have their .META. table updated with the right MEMSTORE_SIZE].
in the shell. This will output the current `.META.` schema. Check `MEMSTORE_FLUSHSIZE` size. Is it 16kb (16384)? If so, you will need to change this (The 'normal'/default value is 64MB (67108864)). Run the script `bin/set_meta_memstore_size.rb`. This will make the necessary edit to your `.META.` schema. Failure to run this change will make for a slow cluster. See link:https://issues.apache.org/jira/browse/HBASE-3499[HBASE-3499 Users upgrading to 0.90.0 need to have their .META. table updated with the right MEMSTORE_SIZE].