blob: 6e8423739b6b668f0f1c9a5ed98b7805b812ae00 [file] [log] [blame]
////
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
////
[[ops_mgt]]
= Apache HBase Operational Management
:doctype: book
:numbered:
:toc: left
:icons: font
: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>>, <<performance>>, and <<configuration>> but is a distinct topic in itself.
[[tools]]
== HBase Tools and Utilities
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.
These are the usage instructions for HBase 0.98.x.
Some commands, such as `version`, `pe`, `ltt`, `clean`, are not available in previous versions.
----
$ bin/hbase
Usage: hbase [<options>] <command> [<args>]
Options:
--config DIR Configuration direction to use. Default: ./conf
--hosts HOSTS Override the list in 'regionservers' file
Commands:
Some commands take arguments. Pass no args or -h for usage.
shell Run the HBase shell
hbck Run the hbase 'fsck' tool
wal Write-ahead-log analyzer
hfile Store file analyzer
zkcli Run the ZooKeeper shell
upgrade Upgrade hbase
master Run an HBase HMaster node
regionserver Run an HBase HRegionServer node
zookeeper Run a ZooKeeper server
rest Run an HBase REST server
thrift Run the HBase Thrift server
thrift2 Run the HBase Thrift2 server
clean Run the HBase clean up script
classpath Dump hbase CLASSPATH
mapredcp Dump CLASSPATH entries required by mapreduce
pe Run PerformanceEvaluation
ltt Run LoadTestTool
version Print the version
CLASSNAME Run the class named CLASSNAME
----
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>>), `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 RegionServer's granularity.
To see the usage, use the `--help` parameter.
----
$ ${HBASE_HOME}/bin/hbase canary -help
Usage: bin/hbase org.apache.hadoop.hbase.tool.Canary [opts] [table1 [table2]...] | [regionserver1 [regionserver2]..]
where [opts] are:
-help Show this help and exit.
-regionserver replace the table argument to regionserver,
which means to enable regionserver mode
-daemon Continuous check at defined intervals.
-interval <N> Interval between checks (sec)
-e Use region/regionserver as regular expression
which means the region/regionserver is regular expression pattern
-f <B> stop whole program if first error occurs, default is true
-t <N> timeout for a check, default is 600000 (milliseconds)
-writeSniffing enable the write sniffing in canary
-treatFailureAsError treats read / write failure as error
-writeTable The table used for write sniffing. Default is hbase:canary
-D<configProperty>=<value> assigning or override the configuration params
----
This tool will return non zero error codes to user for collaborating with other monitoring tools, such as Nagios.
The error code definitions are:
[source,java]
----
private static final int USAGE_EXIT_CODE = 1;
private static final int INIT_ERROR_EXIT_CODE = 2;
private static final int TIMEOUT_ERROR_EXIT_CODE = 3;
private static final int ERROR_EXIT_CODE = 4;
----
Here are some examples based on the following given case.
There are two Table objects 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
|===
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 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
13/12/09 03:26:32 INFO tool.Canary: read from region test-01,0004883,1386230156732.87b55e03dfeade00f441125159f8ca87. column family cf2 in 1ms
...
13/12/09 03:26:32 INFO tool.Canary: read from region test-02,,1386559511167.aa2951a86289281beee480f107bb36ee. column family cf1 in 5ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-02,,1386559511167.aa2951a86289281beee480f107bb36ee. column family cf2 in 3ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-02,0004883,1386559511167.cbda32d5e2e276520712d84eaaa29d84. column family cf1 in 31ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-02,0004883,1386559511167.cbda32d5e2e276520712d84eaaa29d84. column family cf2 in 8ms
----
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.
==== 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 canary test-01 test-02
----
==== Canary test with RegionServer granularity
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 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
----
==== Canary test with regular expression pattern
This will test both table test-01 and test-02.
----
$ ${HBASE_HOME}/bin/hbase 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.
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 canary -daemon
----
Run repeatedly with internal 5 seconds and will not stop itself even if errors occur in the test.
----
$ ${HBASE_HOME}/bin/hbase canary -daemon -interval 50000 -f false
----
==== Force timeout if canary test stuck
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 canary -t 600000
----
==== Enable write sniffing in canary
By default, the canary tool only check the read operations, it's hard to find the problem in the
write path. To enable the write sniffing, you can run canary with the `-writeSniffing` option.
When the write sniffing is enabled, the canary tool will create an hbase table and make sure the
regions of the table distributed on all region servers. In each sniffing period, the canary will
try to put data to these regions to check the write availability of each region server.
----
$ ${HBASE_HOME}/bin/hbase canary -writeSniffing
----
The default write table is `hbase:canary` and can be specified by the option `-writeTable`.
----
$ ${HBASE_HOME}/bin/hbase canary -writeSniffing -writeTable ns:canary
----
The default value size of each put is 10 bytes and you can set it by the config key:
`hbase.canary.write.value.size`.
==== Treat read / write failure as error
By default, the canary tool only logs read failure, due to e.g. RetriesExhaustedException,
while returning normal exit code. To treat read / write failure as error, you can run canary
with the `-treatFailureAsError` option. When enabled, read / write failure would result in error
exit code.
----
$ ${HBASE_HOME}/bin/hbase canary --treatFailureAsError
----
==== Running Canary in a Kerberos-enabled Cluster
To run Canary in a Kerberos-enabled cluster, configure the following two properties in _hbase-site.xml_:
* `hbase.client.keytab.file`
* `hbase.client.kerberos.principal`
Kerberos credentials are refreshed every 30 seconds when Canary runs in daemon mode.
To configure the DNS interface for the client, configure the following optional properties in _hbase-site.xml_.
* `hbase.client.dns.interface`
* `hbase.client.dns.nameserver`
.Canary in a Kerberos-Enabled Cluster
====
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>
</property>
<property>
<name>hbase.client.keytab.file</name>
<value>/etc/hbase/conf/keytab.krb5</value>
</property>
<!-- optional params -->
property>
<name>hbase.client.dns.interface</name>
<value>default</value>
</property>
<property>
<name>hbase.client.dns.nameserver</name>
<value>default</value>
</property>
----
====
[[health.check]]
=== Health Checker
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 _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.
----
${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.mapreduce.UtilityName
----
The following utilities are available:
`LoadIncrementalHFiles`::
Complete a bulk data load.
`CopyTable`::
Export a table from the local cluster to a peer cluster.
`Export`::
Write table data to HDFS.
`Import`::
Import data written by a previous `Export` operation.
`ImportTsv`::
Import data in TSV format.
`RowCounter`::
Count rows in an HBase table.
`CellCounter`::
Count cells in an HBase table.
`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` and `CellCounter` accept a single `--help` argument to print usage instructions.
[[hbck]]
=== HBase `hbck`
To run `hbck` against your HBase cluster run `$./bin/hbase hbck`. At the end of the command's output it prints `OK` or `INCONSISTENCY`.
If your cluster reports inconsistencies, pass `-details` to see more detail emitted.
If inconsistencies, run `hbck` a few times because the inconsistency may be transient (e.g. cluster is starting up or a region is splitting).
Passing `-fix` may correct the inconsistency (This is an experimental feature).
For more information, see <<hbck.in.depth>>.
[[hfile_tool2]]
=== HFile Tool
See <<hfile_tool>>.
=== WAL Tools
[[hlog_tool]]
==== `FSHLog` tool
The main method on `FSHLog` offers manual split and dump facilities.
Pass it WALs or the product of a split, the content of the _recovered.edits_.
directory.
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 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:
----
$ ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.FSHLog --split hdfs://example.org:8020/hbase/.logs/example.org,60020,1283516293161/
----
[[hlog_tool.prettyprint]]
===== 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.
----
$ ./bin/hbase wal hdfs://example.org:8020/hbase/.logs/example.org,60020,1283516293161/10.10.21.10%3A60020.1283973724012
----
.WAL Printing in older versions of HBase
[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 print 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
----
====
[[compression.tool]]
=== Compression Tool
See <<compression.test,compression.test>>.
[[copy.table]]
=== CopyTable
CopyTable is a utility that can copy part or of all of a table, either to the same cluster or another cluster.
The target table must first exist.
The usage is as follows:
----
$ ./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,
specify if different from current cluster
rs.impl hbase.regionserver.impl of the peer cluster,
startrow the start row
stoprow the stop row
starttime beginning of the time range (unixtime in millis)
without endtime means from starttime to forever
endtime end of the time range. Ignored if no starttime specified.
versions number of cell versions to copy
new.name new table's name
peer.adr Address of the peer cluster given in the format
hbase.zookeeer.quorum:hbase.zookeeper.client.port:zookeeper.znode.parent
families comma-separated list of families to copy
To copy from cf1 to cf2, give sourceCfName:destCfName.
To keep the same name, just give "cfName"
all.cells also copy delete markers and deleted cells
Args:
tablename Name of the table to copy
Examples:
To copy 'TestTable' to a cluster that uses replication for a 1 hour window:
$ bin/hbase org.apache.hadoop.hbase.mapreduce.CopyTable --starttime=1265875194289 --endtime=1265878794289 --peer.adr=server1,server2,server3:2181:/hbase --families=myOldCf:myNewCf,cf2,cf3 TestTable
For performance consider the following general options:
It is recommended that you set the following to >=100. A higher value uses more memory but
decreases the round trip time to the server and may increase performance.
-Dhbase.client.scanner.caching=100
The following should always be set to false, to prevent writing data twice, which may produce
inaccurate results.
-Dmapred.map.tasks.speculative.execution=false
----
.Scanner Caching
[NOTE]
====
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.
====
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`.
[[export]]
=== Export
Export is a utility that will dump the contents of table to HDFS in a sequence file.
Invoke via:
----
$ bin/hbase org.apache.hadoop.hbase.mapreduce.Export <tablename> <outputdir> [<versions> [<starttime> [<endtime>]]]
----
NOTE: To see usage instructions, run the command with no options. Available options include
specifying column families and applying filters during the export.
By default, the `Export` tool only exports the newest version of a given cell, regardless of the number of versions stored. To export more than one version, replace *_<versions>_* with the desired number of versions.
Note: caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration.
[[import]]
=== Import
Import is a utility that will load data that has been exported back into HBase.
Invoke via:
----
$ bin/hbase org.apache.hadoop.hbase.mapreduce.Import <tablename> <inputdir>
----
NOTE: To see usage instructions, run the command with no options.
To import 0.94 exported files in a 0.96 cluster or onwards, you need to set system property "hbase.import.version" when running the import command as below:
----
$ bin/hbase -Dhbase.import.version=0.94 org.apache.hadoop.hbase.mapreduce.Import <tablename> <inputdir>
----
[[importtsv]]
=== 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`.
To load data via Puts (i.e., non-bulk loading):
----
$ bin/hbase org.apache.hadoop.hbase.mapreduce.ImportTsv -Dimporttsv.columns=a,b,c <tablename> <hdfs-inputdir>
----
To generate StoreFiles for bulk-loading:
[source,bourne]
----
$ 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>>.
[[importtsv.options]]
==== ImportTsv Options
Running `ImportTsv` with no arguments prints brief usage information:
----
Usage: importtsv -Dimporttsv.columns=a,b,c <tablename> <inputdir>
Imports the given input directory of TSV data into the specified table.
The column names of the TSV data must be specified using the -Dimporttsv.columns
option. This option takes the form of comma-separated column names, where each
column name is either a simple column family, or a columnfamily:qualifier. The special
column name HBASE_ROW_KEY is used to designate that this column should be used
as the row key for each imported record. You must specify exactly one column
to be the row key, and you must specify a column name for every column that exists in the
input data.
By default importtsv will load data directly into HBase. To instead generate
HFiles of data to prepare for a bulk data load, pass the option:
-Dimporttsv.bulk.output=/path/for/output
Note: the target table will be created with default column family descriptors if it does not already exist.
Other options that may be specified with -D include:
-Dimporttsv.skip.bad.lines=false - fail if encountering an invalid line
'-Dimporttsv.separator=|' - eg separate on pipes instead of tabs
-Dimporttsv.timestamp=currentTimeAsLong - use the specified timestamp for the import
-Dimporttsv.mapper.class=my.Mapper - A user-defined Mapper to use instead of org.apache.hadoop.hbase.mapreduce.TsvImporterMapper
----
[[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".
Assume that an input file exists as follows:
----
row1 c1 c2
row2 c1 c2
row3 c1 c2
row4 c1 c2
row5 c1 c2
row6 c1 c2
row7 c1 c2
row8 c1 c2
row9 c1 c2
row10 c1 c2
----
For ImportTsv to use this input file, the command line needs to look like this:
----
HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` ${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/hbase-server-VERSION.jar importtsv -Dimporttsv.columns=HBASE_ROW_KEY,d:c1,d:c2 -Dimporttsv.bulk.output=hdfs://storefileoutput datatsv hdfs://inputfile
----
\... 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.
[[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.
[[importtsv.also]]
==== See Also
For more information about bulk-loading HFiles into HBase, see <<arch.bulk.load,arch.bulk.load>>
[[completebulkload]]
=== CompleteBulkLoad
The `completebulkload` utility will move generated StoreFiles into an HBase table.
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:
.Explicit Classname
----
$ bin/hbase org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles <hdfs://storefileoutput> <tablename>
----
.Driver
----
HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` ${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/hbase-server-VERSION.jar completebulkload <hdfs://storefileoutput> <tablename>
----
[[completebulkload.warning]]
==== CompleteBulkLoad Warning
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>>.
=== WALPlayer
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.
WALPlayer can also generate HFiles for later bulk importing, in that case only a single table and no mapping can be specified.
Invoke via:
----
$ bin/hbase org.apache.hadoop.hbase.mapreduce.WALPlayer [options] <wal inputdir> <tables> [<tableMappings>]>
----
For example:
----
$ bin/hbase org.apache.hadoop.hbase.mapreduce.WALPlayer /backuplogdir oldTable1,oldTable2 newTable1,newTable2
----
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.
[[rowcounter]]
=== RowCounter and CellCounter
link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html[RowCounter] is a mapreduce job to count all the rows of a table.
This is a good utility to use as a sanity check to ensure that HBase can read all the blocks of a table if there are any concerns of metadata inconsistency.
It will run the mapreduce all in a single process but it will run faster if you have a MapReduce cluster in place for it to exploit. It is also possible to limit
the time range of data to be scanned by using the `--starttime=[starttime]` and `--endtime=[endtime]` flags.
----
$ bin/hbase org.apache.hadoop.hbase.mapreduce.RowCounter <tablename> [<column1> <column2>...]
----
RowCounter only counts one version per cell.
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:
* Total number of rows in the table.
* Total number of CFs across all rows.
* Total qualifiers across all rows.
* Total occurrence of each CF.
* Total occurrence of each qualifier.
* Total number of versions of each qualifier.
The program allows you to limit the scope of the run.
Provide a row regex or prefix to limit the rows to analyze.
Specify a time range to scan the table by using the `--starttime=[starttime]` and `--endtime=[endtime]` flags.
Use `hbase.mapreduce.scan.column.family` to specify scanning a single column family.
----
$ 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.
=== 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.
[[compaction.tool]]
=== Offline Compaction Tool
See the usage for the
link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/CompactionTool.html[CompactionTool].
Run it like:
[source, bash]
----
$ ./bin/hbase org.apache.hadoop.hbase.regionserver.CompactionTool
----
=== `hbase clean`
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.
----
$ bin/hbase clean
Usage: hbase clean (--cleanZk|--cleanHdfs|--cleanAll)
Options:
--cleanZk cleans hbase related data from zookeeper.
--cleanHdfs cleans hbase related data from hdfs.
--cleanAll cleans hbase related data from both zookeeper and hdfs.
----
=== `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 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`.
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`
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.
To run LoadTestTool prior to HBase 0.98.4, issue the command +hbase
org.apache.hadoop.hbase.util.LoadTestTool+.
The LoadTestTool has received many updates in recent HBase releases, including support for namespaces, support for tags, cell-level ACLS and visibility labels, testing security-related features, ability to specify the number of regions per server, tests for multi-get RPC calls, and tests relating to replication.
[[ops.regionmgt]]
== Region Management
[[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/Admin.html#majorCompact%28java.lang.String%29[Admin.majorCompact].
Note: major compactions do NOT do region merges.
See <<compaction,compaction>> for more information about compactions.
[[ops.regionmgt.merge]]
=== Merge
Merge is a utility that can merge adjoining regions in the same table (see org.apache.hadoop.hbase.util.Merge).
[source,bourne]
----
$ 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.
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.
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
[[decommission]]
=== Node Decommission
You can stop an individual RegionServer by running the following script in the HBase directory on the particular node:
----
$ ./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.
.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.
====
.Kill Node Tool
[NOTE]
====
In hbase-2.0, in the bin directory, we added a script named _considerAsDead.sh_ that can be used to kill a regionserver.
Hardware issues could be detected by specialized monitoring tools before the zookeeper timeout has expired. _considerAsDead.sh_ is a simple function to mark a RegionServer as dead.
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.
====
A downside to the above stop of a RegionServer is that regions could be offline for a good period of time.
Regions are closed in order.
If many regions on the server, the first region to close may not be back online until all regions close and after the master notices the RegionServer's znode gone.
In Apache HBase 0.90.2, we added facility for having a node gradually shed its load and then shutdown itself down.
Apache HBase 0.90.2 added the _graceful_stop.sh_ script.
Here is its usage:
----
$ ./bin/graceful_stop.sh
Usage: graceful_stop.sh [--config &conf-dir>] [--restart] [--reload] [--thrift] [--rest] &hostname>
thrift If we should stop/start thrift before/after the hbase stop/start
rest If we should stop/start rest before/after the hbase stop/start
restart If we should restart after graceful stop
reload Move offloaded regions back on to the stopped server
debug Move offloaded regions back on to the stopped server
hostname Hostname of server we are to stop
----
To decommission a loaded RegionServer, run the following: +$
./bin/graceful_stop.sh HOSTNAME+ where `HOSTNAME` is the host carrying the RegionServer you would decommission.
.On `HOSTNAME`
[NOTE]
====
The `HOSTNAME` passed to _graceful_stop.sh_ must match the hostname that hbase is using to identify RegionServers.
Check the list of RegionServers in the master UI for how HBase is referring to servers.
It's 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.
====
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.
[[lb]]
.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:
[source]
----
hbase(main):001:0> balance_switch false
true
0 row(s) in 0.3590 seconds
----
This turns the balancer OFF.
To reenable, do:
[source]
----
hbase(main):001:0> balance_switch true
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.
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.
====
[[draining.servers]]
==== Decommissioning several Regions Servers concurrently
If you have a large cluster, you may want to decommission more than one machine at a time by gracefully stopping multiple 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.
Without this facility, decommissioning multiple 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.
See this link:http://inchoate-clatter.blogspot.com/2012/03/hbase-ops-automation.html[blog
post] for more details.
[[bad.disk]]
==== Bad or Failing Disk
It is good having <<dfs.datanode.failed.volumes.tolerated,dfs.datanode.failed.volumes.tolerated>> set if you have a decent number of disks per machine for the case where a disk plain dies.
But usually disks do the "John Wayne" -- i.e.
take a while to go down spewing errors in _dmesg_ -- or for some reason, run much slower than their companions.
In this case you want to decommission the disk.
You have two options.
You can link: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.
.Short Circuit Reads
[NOTE]
====
If you are doing short-circuit reads, you will have to move the regions off the regionserver before you stop the datanode; when short-circuiting reading, though chmod'd so regionserver cannot have access, because it already has the files open, it will be able to keep reading the file blocks from the bad disk even though the datanode is down.
Move the regions back after you restart the datanode.
====
[[rolling]]
=== Rolling Restart
Some cluster configuration changes require either the entire cluster, or the RegionServers, to be restarted in order to pick up the changes.
In addition, rolling restarts are supported for upgrading to a minor or maintenance release, and to a major release if at all possible.
See the release notes for release you want to upgrade to, to find out about limitations to the ability to perform a rolling upgrade.
There are multiple ways to restart your cluster nodes, depending on your situation.
These methods are detailed below.
==== 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.
It requires password-less SSH login to be configured and assumes that you have deployed using a tarball.
The script requires you to set some environment variables before running it.
Examine the script and modify it to suit your needs.
._rolling-restart.sh_ General Usage
====
----
$ ./bin/rolling-restart.sh --help
Usage: rolling-restart.sh [--config <hbase-confdir>] [--rs-only] [--master-only] [--graceful] [--maxthreads xx]
----
====
Rolling Restart on RegionServers Only::
To perform a rolling restart on the RegionServers only, use the `--rs-only` option.
This might be necessary if you need to reboot the individual RegionServer or if you make a configuration change that only affects RegionServers and not the other HBase processes.
Rolling Restart on Masters Only::
To perform a rolling restart on the active and backup Masters, use the `--master-only` option.
You might use this if you know that your configuration change only affects the Master and not the RegionServers, or if you need to restart the server where the active Master is running.
Graceful Restart::
If you specify the `--graceful` option, RegionServers are restarted using the _bin/graceful_stop.sh_ script, which moves regions off a RegionServer before restarting it.
This is safer, but can delay the restart.
Limiting the Number of Threads::
To limit the rolling restart to using only a specific number of threads, use the `--maxthreads` option.
[[rolling.restart.manual]]
==== 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>>.
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.
You may need to tailor it to your environment.
This script does a rolling restart of RegionServers only.
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.
==== 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.
. Use the hbck utility to ensure that the cluster is consistent.
+
----
$ ./bin/hbck
----
+
Perform repairs if required.
See <<hbck,hbck>> for details.
. Restart the master first.
You may need to modify these commands if your new HBase directory is different from the old one, such as for an upgrade.
+
----
$ ./bin/hbase-daemon.sh stop master; ./bin/hbase-daemon.sh start master
----
. Gracefully restart each RegionServer, using a script such as the following, from the Master.
+
----
$ for i in `cat conf/regionservers|sort`; do ./bin/graceful_stop.sh --restart --reload --debug $i; done &> /tmp/log.txt &
----
+
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.
+
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`.
To wait for 5 minutes between each RegionServer restart, modify the above script to the following:
+
----
$ for i in `cat conf/regionservers|sort`; do ./bin/graceful_stop.sh --restart --reload --debug $i & sleep 5m; done &> /tmp/log.txt &
----
. Restart the Master again, to clear out the dead servers list and re-enable the load balancer.
. Run the `hbck` utility again, to be sure the cluster is consistent.
[[adding.new.node]]
=== Adding a New Node
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.
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).
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.
[[hbase_metrics]]
== HBase Metrics
HBase emits metrics which adhere to the link:http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html[Hadoop metrics] API.
Starting with HBase 0.95footnote:[The Metrics system was redone in
HBase 0.96. See Migration
to the New Metrics Hotness – Metrics2 by Elliot Clark for detail], HBase is configured to emit a default set of metrics with a default sampling period of every 10 seconds.
You can use HBase metrics in conjunction with Ganglia.
You can also filter which metrics are emitted and extend the metrics framework to capture custom metrics appropriate for your environment.
=== Metric Setup
For HBase 0.95 and newer, HBase ships with a default metrics configuration, or [firstterm]_sink_.
This includes a wide variety of individual metrics, and emits them every 10 seconds by default.
To configure metrics for a given region server, edit the _conf/hadoop-metrics2-hbase.properties_ file.
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 http://hadoop.apache.org/docs/current/api/org/apache/hadoop/metrics2/package-summary.html
.HBase Metrics and Ganglia
[NOTE]
====
By default, HBase emits a large number of metrics per region server.
Ganglia may have difficulty processing all these metrics.
Consider increasing the capacity of the Ganglia server or reducing the number of metrics emitted by HBase.
See link:http://hadoop.apache.org/docs/current/api/org/apache/hadoop/metrics2/package-summary.html#filtering[Metrics Filtering].
====
=== Disabling Metrics
To disable metrics for a region server, edit the _conf/hadoop-metrics2-hbase.properties_ file and comment out any uncommented lines.
Restart the region server for the changes to take effect.
[[discovering.available.metrics]]
=== Discovering Available Metrics
Rather than listing each metric which HBase emits by default, you can browse through the available metrics, either as a JSON output or via JMX.
Different metrics are exposed for the Master process and each region server process.
.Procedure: Access a JSON Output of Available Metrics
. After starting HBase, access the region server's web UI, at pass:[http://REGIONSERVER_HOSTNAME:60030] by default (or port 16030 in HBase 1.0+).
. Click the [label]#Metrics Dump# link near the top.
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 pass:[http://REGIONSERVER_HOSTNAME:60030/jmx?description=true].
Not all beans and attributes have descriptions.
. To view metrics for the Master, connect to the Master's web UI instead (defaults to pass:[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 pass:[http://REGIONSERVER_HOSTNAME:60010/jmx?description=true].
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.
.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.
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].
When finished, click btn:[Close].
. To view details for a given HBase process, double-click the process in the [label]#Local# sub-tree in the left-hand panel.
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:
. 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.
=== Units of Measure for Metrics
Different metrics are expressed in different units, as appropriate.
Often, the unit of measure is in the name (as in the metric `shippedKBs`). Otherwise, use the following guidelines.
When in doubt, you may need to examine the source for a given metric.
* Metrics that refer to a point in time are usually expressed as a timestamp.
* Metrics that refer to an age (such as `ageOfLastShippedOp`) are usually expressed in milliseconds.
* Metrics that refer to memory sizes are in bytes.
* Sizes of queues (such as `sizeOfLogQueue`) are expressed as the number of items in the queue.
Determine the size by multiplying by the block size (default is 64 MB in HDFS).
* Metrics that refer to things like the number of a given type of operations (such as `logEditsRead`) are expressed as an integer.
[[master_metrics]]
=== Most Important Master Metrics
Note: Counts are usually over the last metrics reporting interval.
hbase.master.numRegionServers::
Number of live regionservers
hbase.master.numDeadRegionServers::
Number of dead regionservers
hbase.master.ritCount ::
The number of regions in transition
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
[[rs_metrics]]
=== Most Important RegionServer Metrics
Note: Counts are usually over the last metrics reporting interval.
hbase.regionserver.regionCount::
The number of regions hosted by the regionserver
hbase.regionserver.storeFileCount::
The number of store files on disk currently managed by the regionserver
hbase.regionserver.storeFileSize::
Aggregate size of the store files on disk
hbase.regionserver.hlogFileCount::
The number of write ahead logs not yet archived
hbase.regionserver.totalRequestCount::
The total number of requests received
hbase.regionserver.readRequestCount::
The number of read requests received
hbase.regionserver.writeRequestCount::
The number of write requests received
hbase.regionserver.numOpenConnections::
The number of open connections at the RPC layer
hbase.regionserver.numActiveHandler::
The number of RPC handlers actively servicing requests
hbase.regionserver.numCallsInGeneralQueue::
The number of currently enqueued user requests
hbase.regionserver.numCallsInReplicationQueue::
The number of currently enqueued operations received from replication
hbase.regionserver.numCallsInPriorityQueue::
The number of currently enqueued priority (internal housekeeping) requests
hbase.regionserver.flushQueueLength::
Current depth of the memstore flush queue.
If increasing, we are falling behind with clearing memstores out to HDFS.
hbase.regionserver.updatesBlockedTime::
Number of milliseconds updates have been blocked so the memstore can be flushed
hbase.regionserver.compactionQueueLength::
Current depth of the compaction request queue.
If increasing, we are falling behind with storefile compaction.
hbase.regionserver.blockCacheHitCount::
The number of block cache hits
hbase.regionserver.blockCacheMissCount::
The number of block cache misses
hbase.regionserver.blockCacheExpressHitPercent ::
The percent of the time that requests with the cache turned on hit the cache
hbase.regionserver.percentFilesLocal::
Percent of store file data that can be read from the local DataNode, 0-100
hbase.regionserver.<op>_<measure>::
Operation latencies, where <op> is one of Append, Delete, Mutate, Get, Replay, Increment; and where <measure> is one of min, max, mean, median, 75th_percentile, 95th_percentile, 99th_percentile
hbase.regionserver.slow<op>Count ::
The number of operations we thought were slow, where <op> is one of the list above
hbase.regionserver.GcTimeMillis::
Time spent in garbage collection, in milliseconds
hbase.regionserver.GcTimeMillisParNew::
Time spent in garbage collection of the young generation, in milliseconds
hbase.regionserver.GcTimeMillisConcurrentMarkSweep::
Time spent in garbage collection of the old generation, in milliseconds
hbase.regionserver.authenticationSuccesses::
Number of client connections where authentication succeeded
hbase.regionserver.authenticationFailures::
Number of client connection authentication failures
hbase.regionserver.mutationsWithoutWALCount ::
Count of writes submitted with a flag indicating they should bypass the write ahead log
[[ops.monitoring]]
== HBase Monitoring
[[ops.monitoring.overview]]
=== 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.
HBase::
* See <<rs_metrics,rs metrics>>
OS::
* IO Wait
* User CPU
Java::
* GC
For more information on HBase metrics, see <<hbase_metrics,hbase metrics>>.
[[ops.slow.query]]
=== Slow Query Log
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 `(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.
* `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.
* `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.
==== Metrics
The slow query log exposes to metrics to JMX.
* `hadoop.regionserver_rpc_slowResponse` a global metric reflecting the durations of all responses that triggered logging.
* `hadoop.regionserver_rpc_methodName.aboveOneSec` A metric reflecting the durations of all responses that lasted for more than one second.
==== Output
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
[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.
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
Starting with HBase 0.98, the HBase Web UI includes the ability to monitor and report on the performance of the block cache.
To view the block cache reports, click .
Following are a few examples of the reporting capabilities.
.Basic Info
image::bc_basic.png[]
.Config
image::bc_config.png[]
.Stats
image::bc_stats.png[]
.L1 and L2
image::bc_l1.png[]
This is not an exhaustive list of all the screens and reports available.
Have a look in the Web UI.
== Cluster Replication
NOTE: This information was previously available at
link:http://hbase.apache.org#replication[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:
* Backup and disaster recovery
* Data aggregation
* Geographic data distribution
* Online data ingestion combined with offline data analytics
NOTE: Replication is enabled at the granularity of the column family.
Before enabling replication for a column family, create the table and all column families to be replicated, on the destination cluster.
=== Replication Overview
Cluster replication uses a source-push methodology.
An HBase cluster can be a source (also called master or active, meaning that it is the originator of new data), a destination (also called slave or passive, meaning that it receives data via replication), or can fulfill both roles at once.
Replication is asynchronous, and the goal of replication is eventual consistency.
When the source receives an edit to a column family with replication enabled, that edit is propagated to all destination clusters using the WAL for that for that column family on the RegionServer managing the relevant region.
When data is replicated from one cluster to another, the original source of the data is tracked via a cluster ID which is part of the metadata.
In HBase 0.96 and newer (link:https://issues.apache.org/jira/browse/HBASE-7709[HBASE-7709]), all clusters which have already consumed the data are also tracked.
This prevents replication loops.
The WALs for each region server must be kept in HDFS as long as they are needed to replicate data to any slave cluster.
Each region server reads from the oldest log it needs to replicate and keeps track of its progress processing WALs inside ZooKeeper to simplify failure recovery.
The position marker which indicates a slave cluster's progress, as well as the queue of WALs to process, may be different for every slave cluster.
The clusters participating in replication can be of different sizes.
The master cluster relies on randomization to attempt to balance the stream of replication on the slave clusters.
It is expected that the slave cluster has storage capacity to hold the replicated data, as well as any data it is responsible for ingesting.
If a slave cluster does run out of room, or is inaccessible for other reasons, it throws an error and the master retains the WAL and retries the replication at intervals.
.Consistency Across Replicated Clusters
[WARNING]
====
How your application builds on top of the HBase API matters when replication is in play. HBase's replication system provides at-least-once delivery of client edits for an enabled column family to each configured destination cluster. In the event of failure to reach a given destination, the replication system will retry sending edits in a way that might repeat a given message. Further more, there is not a guaranteed order of delivery for client edits. In the event of a RegionServer failing, recovery of the replication queue happens independent of recovery of the individual regions that server was previously handling. This means that it is possible for the not-yet-replicated edits to be serviced by a RegionServer that is currently slower to replicate than the one that handles edits from after the failure.
The combination of these two properties (at-least-once delivery and the lack of message ordering) means that some destination clusters may end up in a different state if your application makes use of operations that are not idempotent, e.g. Increments.
====
.Terminology Changes
[NOTE]
====
Previously, terms such as [firstterm]_master-master_, [firstterm]_master-slave_, and [firstterm]_cyclical_ were used to describe replication relationships in HBase.
These terms added confusion, and have been abandoned in favor of discussions about cluster topologies appropriate for different scenarios.
====
.Cluster Topologies
* A central source cluster might propagate changes out to multiple destination clusters, for failover or due to geographic distribution.
* A source cluster might push changes to a destination cluster, which might also push its own changes back to the original cluster.
* Many different low-latency clusters might push changes to one centralized cluster for backup or resource-intensive data analytics jobs.
The processed data might then be replicated back to the low-latency clusters.
Multiple levels of replication may be chained together to suit your organization's needs.
The following diagram shows a hypothetical scenario.
Use the arrows to follow the data paths.
.Example of a Complex Cluster Replication Configuration
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.
=== Managing and Configuring Cluster Replication
.Cluster Configuration Overview
. Configure and start the source and destination clusters.
Create tables with the same names and column families on both the source and destination clusters, so that the destination cluster knows where to store data it will receive.
. All hosts in the source and destination clusters should be reachable to each other.
. If both clusters use the same ZooKeeper cluster, you must use a different `zookeeper.znode.parent`, because they cannot write in the same folder.
. Check to be sure that replication has not been disabled. `hbase.replication` defaults to `true`.
. On the source cluster, in HBase Shell, add the destination cluster as a peer, using the `add_peer` command.
. On the source cluster, in HBase Shell, enable the table replication, using the `enable_table_replication` command.
. Check the logs to see if replication is taking place. If so, you will see messages like the following, coming from the ReplicationSource.
----
LOG.info("Replicating "+clusterId + " -> " + peerClusterId);
----
.Cluster Management Commands
add_peer <ID> <CLUSTER_KEY>::
Adds a replication relationship between two clusters. +
* ID -- a unique string, which must not contain a hyphen.
* CLUSTER_KEY: composed using the following template, with appropriate place-holders: `hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent`
list_peers:: list all replication relationships known by this cluster
enable_peer <ID>::
Enable a previously-disabled replication relationship
disable_peer <ID>::
Disable a replication relationship. HBase will no longer send edits to that
peer cluster, but it still keeps track of all the new WALs that it will need
to replicate if and when it is re-enabled. WALs are retained when enabling or disabling
replication as long as peers exist.
remove_peer <ID>::
Disable and remove a replication relationship. HBase will no longer send edits to that peer cluster or keep track of WALs.
enable_table_replication <TABLE_NAME>::
Enable the table replication switch for all its column families. If the table is not found in the destination cluster then it will create one with the same name and column families.
disable_table_replication <TABLE_NAME>::
Disable the table replication switch for all its column families.
=== Verifying Replicated Data
The `VerifyReplication` MapReduce job, which is included in HBase, performs a systematic comparison of replicated data between two different clusters. Run the VerifyReplication job on the master cluster, supplying it with the peer ID and table name to use for validation. You can limit the verification further by specifying a time range or specific families. The job's short name is `verifyrep`. To run the job, use a command like the following:
+
[source,bash]
----
$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` "${HADOOP_HOME}/bin/hadoop" jar "${HBASE_HOME}/hbase-server-VERSION.jar" verifyrep --starttime=<timestamp> --endtime=<timestamp> --families=<myFam> <ID> <tableName>
----
+
The `VerifyReplication` command prints out `GOODROWS` and `BADROWS` counters to indicate rows that did and did not replicate correctly.
=== Detailed Information About Cluster Replication
.Replication Architecture Overview
image::replication_overview.png[]
==== Life of a WAL Edit
A single WAL edit goes through several steps in order to be replicated to a slave cluster.
. An HBase client uses a Put or Delete operation to manipulate data in HBase.
. The region server writes the request to the WAL in a way allows it to be replayed if it is not written successfully.
. If the changed cell corresponds to a column family that is scoped for replication, the edit is added to the queue for replication.
. In a separate thread, the edit is read from the log, as part of a batch process.
Only the KeyValues that are eligible for replication are kept.
Replicable KeyValues are part of a column family whose schema is scoped GLOBAL, are not part of a catalog such as `hbase:meta`, did not originate from the target slave cluster, and have not already been consumed by the target slave cluster.
. The edit is tagged with the master's UUID and added to a buffer.
When the buffer is filled, or the reader reaches the end of the file, the buffer is sent to a random region server on the slave cluster.
. The region server reads the edits sequentially and separates them into buffers, one buffer per table.
After all edits are read, each buffer is flushed using link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table], HBase's normal client.
The master's UUID and the UUIDs of slaves which have already consumed the data are preserved in the edits they are applied, in order to prevent replication loops.
. In the master, the offset for the WAL that is currently being replicated is registered in ZooKeeper.
. The first three steps, where the edit is inserted, are identical.
. Again in a separate thread, the region server reads, filters, and edits the log edits in the same way as above.
The slave region server does not answer the RPC call.
. The master sleeps and tries again a configurable number of times.
. If the slave region server is still not available, the master selects a new subset of region server to replicate to, and tries again to send the buffer of edits.
. Meanwhile, the WALs are rolled and stored in a queue in ZooKeeper.
Logs that are [firstterm]_archived_ by their region server, by moving them from the region server's log directory to a central log directory, will update their paths in the in-memory queue of the replicating thread.
. When the slave cluster is finally available, the buffer is applied in the same way as during normal processing.
The master region server will then replicate the backlog of logs that accumulated during the outage.
.Spreading Queue Failover Load
When replication is active, a subset of region servers in the source cluster is responsible for shipping edits to the sink.
This responsibility must be failed over like all other region server functions should a process or node crash.
The following configuration settings are recommended for maintaining an even distribution of replication activity over the remaining live servers in the source cluster:
* Set `replication.source.maxretriesmultiplier` to `300`.
* Set `replication.source.sleepforretries` to `1` (1 second). This value, combined with the value of `replication.source.maxretriesmultiplier`, causes the retry cycle to last about 5 minutes.
* Set `replication.sleep.before.failover` to `30000` (30 seconds) in the source cluster site configuration.
[[cluster.replication.preserving.tags]]
.Preserving Tags During Replication
By default, the codec used for replication between clusters strips tags, such as cell-level ACLs, from cells.
To prevent the tags from being stripped, you can use a different codec which does not strip them.
Configure `hbase.replication.rpc.codec` to use `org.apache.hadoop.hbase.codec.KeyValueCodecWithTags`, on both the source and sink RegionServers involved in the replication.
This option was introduced in link:https://issues.apache.org/jira/browse/HBASE-10322[HBASE-10322].
==== Replication Internals
Replication State in ZooKeeper::
HBase replication maintains its state in ZooKeeper.
By default, the state is contained in the base node _/hbase/replication_.
This node contains two child nodes, the `Peers` znode and the `RS` znode.
The `Peers` Znode::
The `peers` znode is stored in _/hbase/replication/peers_ by default.
It consists of a list of all peer replication clusters, along with the status of each of them.
The value of each peer is its cluster key, which is provided in the HBase Shell.
The cluster key contains a list of ZooKeeper nodes in the cluster's quorum, the client port for the ZooKeeper quorum, and the base znode for HBase in HDFS on that cluster.
The `RS` Znode::
The `rs` znode contains a list of WAL logs which need to be replicated.
This list is divided into a set of queues organized by region server and the peer cluster the region server is shipping the logs to.
The rs znode has one child znode for each region server in the cluster.
The child znode name is the region server's hostname, client port, and start code.
This list includes both live and dead region servers.
==== Choosing Region Servers to Replicate To
When a master cluster region server initiates a replication source to a slave cluster, it first connects to the slave's ZooKeeper ensemble using the provided cluster key . It then scans the _rs/_ directory to discover all the available sinks (region servers that are accepting incoming streams of edits to replicate) and randomly chooses a subset of them using a configured ratio which has a default value of 10%. For example, if a slave cluster has 150 machines, 15 will be chosen as potential recipient for edits that this master cluster region server sends.
Because this selection is performed by each master region server, the probability that all slave region servers are used is very high, and this method works for clusters of any size.
For example, a master cluster of 10 machines replicating to a slave cluster of 5 machines with a ratio of 10% causes the master cluster region servers to choose one machine each at random.
A ZooKeeper watcher is placed on the _${zookeeper.znode.parent}/rs_ node of the slave cluster by each of the master cluster's region servers.
This watch is used to monitor changes in the composition of the slave cluster.
When nodes are removed from the slave cluster, or if nodes go down or come back up, the master cluster's region servers will respond by selecting a new pool of slave region servers to replicate to.
==== Keeping Track of Logs
Each master cluster region server has its own znode in the replication znodes hierarchy.
It contains one znode per peer cluster (if 5 slave clusters, 5 znodes are created), and each of these contain a queue of WALs to process.
Each of these queues will track the WALs created by that region server, but they can differ in size.
For example, if one slave cluster becomes unavailable for some time, the WALs should not be deleted, so they need to stay in the queue while the others are processed.
See <<rs.failover.details,rs.failover.details>> for an example.
When a source is instantiated, it contains the current WAL that the region server is writing to.
During log rolling, the new file is added to the queue of each slave cluster's znode just before it is made available.
This ensures that all the sources are aware that a new log exists before the region server is able to append edits into it, but this operations is now more expensive.
The queue items are discarded when the replication thread cannot read more entries from a file (because it reached the end of the last block) and there are other files in the queue.
This means that if a source is up to date and replicates from the log that the region server writes to, reading up to the "end" of the current file will not delete the item in the queue.
A log can be archived if it is no longer used or if the number of logs exceeds `hbase.regionserver.maxlogs` because the insertion rate is faster than regions are flushed.
When a log is archived, the source threads are notified that the path for that log changed.
If a particular source has already finished with an archived log, it will just ignore the message.
If the log is in the queue, the path will be updated in memory.
If the log is currently being replicated, the change will be done atomically so that the reader doesn't attempt to open the file when has already been moved.
Because moving a file is a NameNode operation , if the reader is currently reading the log, it won't generate any exception.
==== Reading, Filtering and Sending Edits
By default, a source attempts to read from a WAL and ship log entries to a sink as quickly as possible.
Speed is limited by the filtering of log entries Only KeyValues that are scoped GLOBAL and that do not belong to catalog tables will be retained.
Speed is also limited by total size of the list of edits to replicate per slave, which is limited to 64 MB by default.
With this configuration, a master cluster region server with three slaves would use at most 192 MB to store data to replicate.
This does not account for the data which was filtered but not garbage collected.
Once the maximum size of edits has been buffered or the reader reaches the end of the WAL, the source thread stops reading and chooses at random a sink to replicate to (from the list that was generated by keeping only a subset of slave region servers). It directly issues a RPC to the chosen region server and waits for the method to return.
If the RPC was successful, the source determines whether the current file has been emptied or it contains more data which needs to be read.
If the file has been emptied, the source deletes the znode in the queue.
Otherwise, it registers the new offset in the log's znode.
If the RPC threw an exception, the source will retry 10 times before trying to find a different sink.
==== Cleaning Logs
If replication is not enabled, the master's log-cleaning thread deletes old logs using a configured TTL.
This TTL-based method does not work well with replication, because archived logs which have exceeded their TTL may still be in a queue.
The default behavior is augmented so that if a log is past its TTL, the cleaning thread looks up every queue until it finds the log, while caching queues it has found.
If the log is not found in any queues, the log will be deleted.
The next time the cleaning process needs to look for a log, it starts by using its cached list.
NOTE: WALs are saved when replication is enabled or disabled as long as peers exist.
[[rs.failover.details]]
==== Region Server Failover
When no region servers are failing, keeping track of the logs in ZooKeeper adds no value.
Unfortunately, region servers do fail, and since ZooKeeper is highly available, it is useful for managing the transfer of the queues in the event of a failure.
Each of the master cluster region servers keeps a watcher on every other region server, in order to be notified when one dies (just as the master does). When a failure happens, they all race to create a znode called `lock` inside the dead region server's znode that contains its queues.
The region server that creates it successfully then transfers all the queues to its own znode, one at a time since ZooKeeper does not support renaming queues.
After queues are all transferred, they are deleted from the old location.
The znodes that were recovered are renamed with the ID of the slave cluster appended with the name of the dead server.
Next, the master cluster region server creates one new source thread per copied queue, and each of the source threads follows the read/filter/ship pattern.
The main difference is that those queues will never receive new data, since they do not belong to their new region server.
When the reader hits the end of the last log, the queue's znode is deleted and the master cluster region server closes that replication source.
Given a master cluster with 3 region servers replicating to a single slave with id `2`, the following hierarchy represents what the znodes layout could be at some point in time.
The region servers' znodes all contain a `peers` znode which contains a single queue.
The znode names in the queues represent the actual file names on HDFS in the form `address,port.timestamp`.
----
/hbase/replication/rs/
1.1.1.1,60020,123456780/
2/
1.1.1.1,60020.1234 (Contains a position)
1.1.1.1,60020.1265
1.1.1.2,60020,123456790/
2/
1.1.1.2,60020.1214 (Contains a position)
1.1.1.2,60020.1248
1.1.1.2,60020.1312
1.1.1.3,60020, 123456630/
2/
1.1.1.3,60020.1280 (Contains a position)
----
Assume that 1.1.1.2 loses its ZooKeeper session.
The survivors will race to create a lock, and, arbitrarily, 1.1.1.3 wins.
It will then start transferring all the queues to its local peers znode by appending the name of the dead server.
Right before 1.1.1.3 is able to clean up the old znodes, the layout will look like the following:
----
/hbase/replication/rs/
1.1.1.1,60020,123456780/
2/
1.1.1.1,60020.1234 (Contains a position)
1.1.1.1,60020.1265
1.1.1.2,60020,123456790/
lock
2/
1.1.1.2,60020.1214 (Contains a position)
1.1.1.2,60020.1248
1.1.1.2,60020.1312
1.1.1.3,60020,123456630/
2/
1.1.1.3,60020.1280 (Contains a position)
2-1.1.1.2,60020,123456790/
1.1.1.2,60020.1214 (Contains a position)
1.1.1.2,60020.1248
1.1.1.2,60020.1312
----
Some time later, but before 1.1.1.3 is able to finish replicating the last WAL from 1.1.1.2, it dies too.
Some new logs were also created in the normal queues.
The last region server will then try to lock 1.1.1.3's znode and will begin transferring all the queues.
The new layout will be:
----
/hbase/replication/rs/
1.1.1.1,60020,123456780/
2/
1.1.1.1,60020.1378 (Contains a position)
2-1.1.1.3,60020,123456630/
1.1.1.3,60020.1325 (Contains a position)
1.1.1.3,60020.1401
2-1.1.1.2,60020,123456790-1.1.1.3,60020,123456630/
1.1.1.2,60020.1312 (Contains a position)
1.1.1.3,60020,123456630/
lock
2/
1.1.1.3,60020.1325 (Contains a position)
1.1.1.3,60020.1401
2-1.1.1.2,60020,123456790/
1.1.1.2,60020.1312 (Contains a position)
----
=== Replication Metrics
The following metrics are exposed at the global region server level and (since HBase 0.95) at the peer level:
`source.sizeOfLogQueue`::
number of WALs to process (excludes the one which is being processed) at the Replication source
`source.shippedOps`::
number of mutations shipped
`source.logEditsRead`::
number of mutations read from WALs at the replication source
`source.ageOfLastShippedOp`::
age of last batch that was shipped by the replication source
=== Replication Configuration Options
[cols="1,1,1", options="header"]
|===
| Option
| Description
| Default
| zookeeper.znode.parent
| The name of the base ZooKeeper znode used for HBase
| /hbase
| zookeeper.znode.replication
| The name of the base znode used for replication
| replication
| zookeeper.znode.replication.peers
| The name of the peer znode
| peers
| zookeeper.znode.replication.peers.state
| The name of peer-state znode
| peer-state
| zookeeper.znode.replication.rs
| The name of the rs znode
| rs
| replication.sleep.before.failover
| How many milliseconds a worker should sleep before attempting to replicate
a dead region server's WAL queues.
|
| replication.executor.workers
| The number of region servers a given region server should attempt to
failover simultaneously.
| 1
|===
=== Monitoring Replication Status
You can use the HBase Shell command `status 'replication'` to monitor the replication status on your cluster. The command has three variations:
* `status 'replication'` -- prints the status of each source and its sinks, sorted by hostname.
* `status 'replication', 'source'` -- prints the status for each replication source, sorted by hostname.
* `status 'replication', 'sink'` -- prints the status for each replication sink, sorted by hostname.
== Running Multiple Workloads On a Single Cluster
HBase provides the following mechanisms for managing the performance of a cluster
handling multiple workloads:
. <<quota>>
. <<request_queues>>
. <<multiple-typed-queues>>
[[quota]]
=== Quotas
HBASE-11598 introduces quotas, which allow you to throttle requests based on
the following limits:
. <<request-quotas,The number or size of requests(read, write, or read+write) in a given timeframe>>
. <<namespace_quotas,The number of tables allowed in a namespace>>
These limits can be enforced for a specified user, table, or namespace.
.Enabling Quotas
Quotas are disabled by default. To enable the feature, set the `hbase.quota.enabled`
property to `true` in _hbase-site.xml_ file for all cluster nodes.
.General Quota Syntax
. THROTTLE_TYPE can be expressed as READ, WRITE, or the default type(read + write).
. Timeframes can be expressed in the following units: `sec`, `min`, `hour`, `day`
. Request sizes can be expressed in the following units: `B` (bytes), `K` (kilobytes),
`M` (megabytes), `G` (gigabytes), `T` (terabytes), `P` (petabytes)
. Numbers of requests are expressed as an integer followed by the string `req`
. Limits relating to time are expressed as req/time or size/time. For instance `10req/day`
or `100P/hour`.
. Numbers of tables or regions are expressed as integers.
[[request-quotas]]
.Setting Request Quotas
You can set quota rules ahead of time, or you can change the throttle at runtime. The change
will propagate after the quota refresh period has expired. This expiration period
defaults to 5 minutes. To change it, modify the `hbase.quota.refresh.period` property
in `hbase-site.xml`. This property is expressed in milliseconds and defaults to `300000`.
----
# Limit user u1 to 10 requests per second
hbase> set_quota TYPE => THROTTLE, USER => 'u1', LIMIT => '10req/sec'
# Limit user u1 to 10 read requests per second
hbase> set_quota TYPE => THROTTLE, THROTTLE_TYPE => READ, USER => 'u1', LIMIT => '10req/sec'
# Limit user u1 to 10 M per day everywhere
hbase> set_quota TYPE => THROTTLE, USER => 'u1', LIMIT => '10M/day'
# Limit user u1 to 10 M write size per sec
hbase> set_quota TYPE => THROTTLE, THROTTLE_TYPE => WRITE, USER => 'u1', LIMIT => '10M/sec'
# Limit user u1 to 5k per minute on table t2
hbase> set_quota TYPE => THROTTLE, USER => 'u1', TABLE => 't2', LIMIT => '5K/min'
# Limit user u1 to 10 read requests per sec on table t2
hbase> set_quota TYPE => THROTTLE, THROTTLE_TYPE => READ, USER => 'u1', TABLE => 't2', LIMIT => '10req/sec'
# Remove an existing limit from user u1 on namespace ns2
hbase> set_quota TYPE => THROTTLE, USER => 'u1', NAMESPACE => 'ns2', LIMIT => NONE
# Limit all users to 10 requests per hour on namespace ns1
hbase> set_quota TYPE => THROTTLE, NAMESPACE => 'ns1', LIMIT => '10req/hour'
# Limit all users to 10 T per hour on table t1
hbase> set_quota TYPE => THROTTLE, TABLE => 't1', LIMIT => '10T/hour'
# Remove all existing limits from user u1
hbase> set_quota TYPE => THROTTLE, USER => 'u1', LIMIT => NONE
# List all quotas for user u1 in namespace ns2
hbase> list_quotas USER => 'u1, NAMESPACE => 'ns2'
# List all quotas for namespace ns2
hbase> list_quotas NAMESPACE => 'ns2'
# List all quotas for table t1
hbase> list_quotas TABLE => 't1'
# list all quotas
hbase> list_quotas
----
You can also place a global limit and exclude a user or a table from the limit by applying the
`GLOBAL_BYPASS` property.
----
hbase> set_quota NAMESPACE => 'ns1', LIMIT => '100req/min' # a per-namespace request limit
hbase> set_quota USER => 'u1', GLOBAL_BYPASS => true # user u1 is not affected by the limit
----
[[namespace_quotas]]
.Setting Namespace Quotas
You can specify the maximum number of tables or regions allowed in a given namespace, either
when you create the namespace or by altering an existing namespace, by setting the
`hbase.namespace.quota.maxtables property` on the namespace.
.Limiting Tables Per Namespace
----
# Create a namespace with a max of 5 tables
hbase> create_namespace 'ns1', {'hbase.namespace.quota.maxtables'=>'5'}
# Alter an existing namespace to have a max of 8 tables
hbase> alter_namespace 'ns2', {METHOD => 'set', 'hbase.namespace.quota.maxtables'=>'8'}
# Show quota information for a namespace
hbase> describe_namespace 'ns2'
# Alter an existing namespace to remove a quota
hbase> alter_namespace 'ns2', {METHOD => 'unset', NAME=>'hbase.namespace.quota.maxtables'}
----
.Limiting Regions Per Namespace
----
# Create a namespace with a max of 10 regions
hbase> create_namespace 'ns1', {'hbase.namespace.quota.maxregions'=>'10'
# Show quota information for a namespace
hbase> describe_namespace 'ns1'
# Alter an existing namespace to have a max of 20 tables
hbase> alter_namespace 'ns2', {METHOD => 'set', 'hbase.namespace.quota.maxregions'=>'20'}
# Alter an existing namespace to remove a quota
hbase> alter_namespace 'ns2', {METHOD => 'unset', NAME=> 'hbase.namespace.quota.maxregions'}
----
[[request_queues]]
=== Request Queues
If no throttling policy is configured, when the RegionServer receives multiple requests,
they are now placed into a queue waiting for a free execution slot (HBASE-6721).
The simplest queue is a FIFO queue, where each request waits for all previous requests in the queue
to finish before running. Fast or interactive queries can get stuck behind large requests.
If you are able to guess how long a request will take, you can reorder requests by
pushing the long requests to the end of the queue and allowing short requests to preempt
them. Eventually, you must still execute the large requests and prioritize the new
requests behind them. The short requests will be newer, so the result is not terrible,
but still suboptimal compared to a mechanism which allows large requests to be split
into multiple smaller ones.
HBASE-10993 introduces such a system for deprioritizing long-running scanners. There
are two types of queues, `fifo` and `deadline`. To configure the type of queue used,
configure the `hbase.ipc.server.callqueue.type` property in `hbase-site.xml`. There
is no way to estimate how long each request may take, so de-prioritization only affects
scans, and is based on the number of “next” calls a scan request has made. An assumption
is made that when you are doing a full table scan, your job is not likely to be interactive,
so if there are concurrent requests, you can delay long-running scans up to a limit tunable by
setting the `hbase.ipc.server.queue.max.call.delay` property. The slope of the delay is calculated
by a simple square root of `(numNextCall * weight)` where the weight is
configurable by setting the `hbase.ipc.server.scan.vtime.weight` property.
[[multiple-typed-queues]]
=== Multiple-Typed Queues
You can also prioritize or deprioritize different kinds of requests by configuring
a specified number of dedicated handlers and queues. You can segregate the scan requests
in a single queue with a single handler, and all the other available queues can service
short `Get` requests.
You can adjust the IPC queues and handlers based on the type of workload, using static
tuning options. This approach is an interim first step that will eventually allow
you to change the settings at runtime, and to dynamically adjust values based on the load.
.Multiple Queues
To avoid contention and separate different kinds of requests, configure the
`hbase.ipc.server.callqueue.handler.factor` property, which allows you to increase the number of
queues and control how many handlers can share the same queue., allows admins to increase the number
of queues and decide how many handlers share the same queue.
Using more queues reduces contention when adding a task to a queue or selecting it
from a queue. You can even configure one queue per handler. The trade-off is that
if some queues contain long-running tasks, a handler may need to wait to execute from that queue
rather than stealing from another queue which has waiting tasks.
.Read and Write Queues
With multiple queues, you can now divide read and write requests, giving more priority
(more queues) to one or the other type. Use the `hbase.ipc.server.callqueue.read.ratio`
property to choose to serve more reads or more writes.
.Get and Scan Queues
Similar to the read/write split, you can split gets and scans by tuning the `hbase.ipc.server.callqueue.scan.ratio`
property to give more priority to gets or to scans. A scan ratio of `0.1` will give
more queue/handlers to the incoming gets, which means that more gets can be processed
at the same time and that fewer scans can be executed at the same time. A value of
`0.9` will give more queue/handlers to scans, so the number of scans executed will
increase and the number of gets will decrease.
[[ops.backup]]
== 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.
For additional information, see link:http://blog.sematext.com/2011/03/11/hbase-backup-options/[HBase Backup
Options] over on the Sematext Blog.
[[ops.backup.fullshutdown]]
=== Full Shutdown Backup
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:
[[ops.backup.fullshutdown.stop]]
==== Stop HBase
[[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.
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.
[[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.
[[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/book.html#replication[replication] for more information.
[[ops.backup.live.copytable]]
=== Live Cluster Backup - CopyTable
The <<copy.table,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.
[[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.
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.
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.
[[ops.snapshots.configuration]]
=== Configuration
To turn on the snapshot support just set the `hbase.snapshot.enabled` property to true.
(Snapshots are enabled by default in 0.95+ and off by default in 0.94.6+)
[source,java]
----
<property>
<name>hbase.snapshot.enabled</name>
<value>true</value>
</property>
----
[[ops.snapshots.takeasnapshot]]
=== Take a Snapshot
You can take a snapshot of a table regardless of whether it is enabled or disabled.
The snapshot operation doesn't involve any data copying.
----
$ ./bin/hbase shell
hbase> snapshot 'myTable', 'myTableSnapshot-122112'
----
.Take a Snapshot Without Flushing
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.
----
hbase> snapshot 'mytable', 'snapshot123', {SKIP_FLUSH => true}
----
WARNING: There is no way to determine or predict whether a very concurrent insert or update will be included in a given snapshot, whether flushing is enabled or disabled.
A snapshot is only a representation of a table during a window of time.
The amount of time the snapshot operation will take to reach each Region Server may vary from a few seconds to a minute, depending on the resource load and speed of the hardware or network, among other factors.
There is also no way to know whether a given insert or update is in memory or has been flushed.
[[ops.snapshots.list]]
=== Listing Snapshots
List all snapshots taken (by printing the names and relative information).
----
$ ./bin/hbase shell
hbase> list_snapshots
----
[[ops.snapshots.delete]]
=== Deleting Snapshots
You can remove a snapshot, and the files retained for that snapshot will be removed if no longer needed.
----
$ ./bin/hbase shell
hbase> delete_snapshot 'myTableSnapshot-122112'
----
[[ops.snapshots.clone]]
=== Clone a table from snapshot
From a snapshot you can create a new table (clone operation) with the same data that you had when the snapshot was taken.
The clone operation, doesn't involve data copies, and a change to the cloned table doesn't impact the snapshot or the original table.
----
$ ./bin/hbase shell
hbase> clone_snapshot 'myTableSnapshot-122112', 'myNewTestTable'
----
[[ops.snapshots.restore]]
=== Restore a snapshot
The restore operation requires the table to be disabled, and the table will be restored to the state at the time when the snapshot was taken, changing both data and schema if required.
----
$ ./bin/hbase shell
hbase> disable 'myTable'
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.
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
If you are using security with the AccessController Coprocessor (See <<hbase.accesscontrol.configuration,hbase.accesscontrol.configuration>>), only a global administrator can take, clone, or restore a snapshot, and these actions do not capture the ACL rights.
This means that restoring a table preserves the ACL rights of the existing table, while cloning a table creates a new table that has no ACL rights until the administrator adds them.
[[ops.snapshots.export]]
=== Export to another cluster
The ExportSnapshot tool copies all the data related to a snapshot (hfiles, logs, snapshot metadata) to another cluster.
The tool executes a Map-Reduce job, similar to distcp, to copy files between the two clusters, and since it works at file-system level the hbase cluster does not have to be online.
To copy a snapshot called MySnapshot to an HBase cluster srv2 (hdfs:///srv2:8082/hbase) using 16 mappers:
[source,bourne]
----
$ bin/hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase -mappers 16
----
.Limiting Bandwidth Consumption
You can limit the bandwidth consumption when exporting a snapshot, by specifying the `-bandwidth` parameter, which expects an integer representing megabytes per second.
The following example limits the above example to 200 MB/sec.
[source,bourne]
----
$ bin/hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase -mappers 16 -bandwidth 200
----
[[snapshots_s3]]
=== Storing Snapshots in an Amazon S3 Bucket
For general information and limitations of using Amazon S3 storage with HBase, see
<<amazon_s3_configuration>>. You can also store and retrieve snapshots from Amazon
S3, using the following procedure.
NOTE: You can also store snapshots in Microsoft Azure Blob Storage. See <<snapshots_azure>>.
.Prerequisites
- You must be using HBase 1.0 or higher and Hadoop 2.6.1 or higher, which is the first
configuration that uses the Amazon AWS SDK.
- You must use the `s3a://` protocol to connect to Amazon S3. The older `s3n://`
and `s3://` protocols have various limitations and do not use the Amazon AWS SDK.
- The `s3a://` URI must be configured and available on the server where you run
the commands to export and restore the snapshot.
After you have fulfilled the prerequisites, take the snapshot like you normally would.
Afterward, you can export it using the `org.apache.hadoop.hbase.snapshot.ExportSnapshot`
command like the one below, substituting your own `s3a://` path in the `copy-from`
or `copy-to` directive and substituting or modifying other options as required:
----
$ hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot \
-snapshot MySnapshot \
-copy-from hdfs://srv2:8082/hbase \
-copy-to s3a://<bucket>/<namespace>/hbase \
-chuser MyUser \
-chgroup MyGroup \
-chmod 700 \
-mappers 16
----
----
$ hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot \
-snapshot MySnapshot
-copy-from s3a://<bucket>/<namespace>/hbase \
-copy-to hdfs://srv2:8082/hbase \
-chuser MyUser \
-chgroup MyGroup \
-chmod 700 \
-mappers 16
----
You can also use the `org.apache.hadoop.hbase.snapshot.SnapshotInfo` utility with the `s3a://` path by including the
`-remote-dir` option.
----
$ hbase org.apache.hadoop.hbase.snapshot.SnapshotInfo \
-remote-dir s3a://<bucket>/<namespace>/hbase \
-list-snapshots
----
[[snapshots_azure]]
== Storing Snapshots in Microsoft Azure Blob Storage
You can store snapshots in Microsoft Azure Blog Storage using the same techniques
as in <<snapshots_s3>>.
.Prerequisites
- You must be using HBase 1.2 or higher with Hadoop 2.7.1 or
higher. No version of HBase supports Hadoop 2.7.0.
- Your hosts must be configured to be aware of the Azure blob storage filesystem.
See http://hadoop.apache.org/docs/r2.7.1/hadoop-azure/index.html.
After you meet the prerequisites, follow the instructions
in <<snapshots_s3>>, replacingthe protocol specifier with `wasb://` or `wasbs://`.
[[ops.capacity]]
== Capacity Planning and Region Sizing
There are several considerations when planning the capacity for an HBase cluster and performing the initial configuration.
Start with a solid understanding of how HBase handles data internally.
[[ops.capacity.nodes]]
=== Node count and hardware/VM configuration
[[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:
* Increased by HBase overhead
+
* See <<keyvalue,keyvalue>> and <<keysize,keysize>>.
At least 24 bytes per key-value (cell), can be more.
Small keys/values means more relative overhead.
* KeyValue instances are aggregated into blocks, which are indexed.
Indexes also have to be stored.
Blocksize is configurable on a per-ColumnFamily basis.
See <<regions.arch,regions.arch>>.
* Decreased by <<compression,compression>> and data block encoding, depending on data.
See also link:http://search-hadoop.com/m/lL12B1PFVhp1[this thread].
You might want to test what compression and encoding (if any) make sense for your data.
* Increased by size of region server <<wal,wal>> (usually fixed and negligible - less than half of RS memory size, per RS).
* Increased by HDFS replication - usually x3.
Aside from the disk space necessary to store the data, one RS may not be able to serve arbitrarily large amounts of data due to some practical limits on region count and size (see <<ops.capacity.regions,ops.capacity.regions>>).
[[ops.capacity.nodes.throughput]]
==== Read/Write throughput
Number of nodes can also be driven by required throughput for reads and/or writes.
The throughput one can get per node depends a lot on data (esp.
key/value sizes) and request patterns, as well as node and system configuration.
Planning should be done for peak load if it is likely that the load would be the main driver of the increase of the node count.
PerformanceEvaluation and <<ycsb,ycsb>> tools can be used to test single node or a test cluster.
For write, usually 5-15Mb/s per RS can be expected, since every region server has only one active WAL.
There's no good estimate for reads, as it depends vastly on data, requests, and cache hit rate. <<perf.casestudy,perf.casestudy>> might be helpful.
[[ops.capacity.nodes.gc]]
==== JVM GC limitations
RS cannot currently utilize very large heap due to cost of GC.
There's also no good way of running multiple RS-es per server (other than running several VMs per machine). Thus, ~20-24Gb or less memory dedicated to one RS is recommended.
GC tuning is required for large heap sizes.
See <<gcpause,gcpause>>, <<trouble.log.gc,trouble.log.gc>> and elsewhere (TODO: where?)
[[ops.capacity.regions]]
=== Determining region count and size
Generally less regions makes for a smoother running cluster (you can always manually split the big regions later (if necessary) to spread the data, or request load, over the cluster); 20-200 regions per RS is a reasonable range.
The number of regions cannot be configured directly (unless you go for fully <<disable.splitting,disable.splitting>>); adjust the region size to achieve the target region size given table size.
When configuring regions for multiple tables, note that most region settings can be set on a per-table basis via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HTableDescriptor.html[HTableDescriptor], as well as shell commands.
These settings will override the ones in `hbase-site.xml`.
That is useful if your tables have different workloads/use cases.
Also note that in the discussion of region sizes here, _HDFS replication factor is not (and should not be) taken into account, whereas
other factors <<ops.capacity.nodes.datasize,ops.capacity.nodes.datasize>> should be._ So, if your data is compressed and replicated 3 ways by HDFS, "9 Gb region" means 9 Gb of compressed data.
HDFS replication factor only affects your disk usage and is invisible to most HBase code.
==== Viewing the Current Number of Regions
You can view the current number of regions for a given table using the HMaster UI.
In the [label]#Tables# section, the number of online regions for each table is listed in the [label]#Online Regions# column.
This total only includes the in-memory state and does not include disabled or offline regions.
If you do not want to use the HMaster UI, you can determine the number of regions by counting the number of subdirectories of the /hbase/<table>/ subdirectories in HDFS, or by running the `bin/hbase hbck` command.
Each of these methods may return a slightly different number, depending on the status of each region.
[[ops.capacity.regions.count]]
==== Number of regions per RS - upper bound
In production scenarios, where you have a lot of data, you are normally concerned with the maximum number of regions you can have per server. <<too_many_regions,too many regions>> has technical discussion on the subject.
Basically, the maximum number of regions is mostly determined by memstore memory usage.
Each region has its own memstores; these grow up to a configurable size; usually in 128-256 MB range, see <<hbase.hregion.memstore.flush.size,hbase.hregion.memstore.flush.size>>.
One memstore exists per column family (so there's only one per region if there's one CF in the table). The RS dedicates some fraction of total memory to its memstores (see <<hbase.regionserver.global.memstore.size,hbase.regionserver.global.memstore.size>>). If this memory is exceeded (too much memstore usage), it can cause undesirable consequences such as unresponsive server or compaction storms.
A good starting point for the number of regions per RS (assuming one table) is:
[source]
----
((RS memory) * (total memstore fraction)) / ((memstore size)*(# column families))
----
This formula is pseudo-code.
Here are two formulas using the actual tunable parameters, first for HBase 0.98+ and second for HBase 0.94.x.
HBase 0.98.x::
----
((RS Xmx) * hbase.regionserver.global.memstore.size) / (hbase.hregion.memstore.flush.size * (# column families))
----
HBase 0.94.x::
----
((RS Xmx) * hbase.regionserver.global.memstore.upperLimit) / (hbase.hregion.memstore.flush.size * (# column families))+
----
If a given RegionServer has 16 GB of RAM, with default settings, the formula works out to 16384*0.4/128 ~ 51 regions per RS is a starting point.
The formula can be extended to multiple tables; if they all have the same configuration, just use the total number of families.
This number can be adjusted; the formula above assumes all your regions are filled at approximately the same rate.
If only a fraction of your regions are going to be actively written to, you can divide the result by that fraction to get a larger region count.
Then, even if all regions are written to, all region memstores are not filled evenly, and eventually jitter appears even if they are (due to limited number of concurrent flushes). Thus, one can have as many as 2-3 times more regions than the starting point; however, increased numbers carry increased risk.
For write-heavy workload, memstore fraction can be increased in configuration at the expense of block cache; this will also allow one to have more regions.
[[ops.capacity.regions.mincount]]
==== Number of regions per RS - lower bound
HBase scales by having regions across many servers.
Thus if you have 2 regions for 16GB data, on a 20 node machine your data will be concentrated on just a few machines - nearly the entire cluster will be idle.
This really can't be stressed enough, since a common problem is loading 200MB data into HBase and then wondering why your awesome 10 node cluster isn't doing anything.
On the other hand, if you have a very large amount of data, you may also want to go for a larger number of regions to avoid having regions that are too large.
[[ops.capacity.regions.size]]
==== Maximum region size
For large tables in production scenarios, maximum region size is mostly limited by compactions - very large compactions, esp.
major, can degrade cluster performance.
Currently, the recommended maximum region size is 10-20Gb, and 5-10Gb is optimal.
For older 0.90.x codebase, the upper-bound of regionsize is about 4Gb, with a default of 256Mb.
The size at which the region is split into two is generally configured via <<hbase.hregion.max.filesize,hbase.hregion.max.filesize>>; for details, see <<arch.region.splits,arch.region.splits>>.
If you cannot estimate the size of your tables well, when starting off, it's probably best to stick to the default region size, perhaps going smaller for hot tables (or manually split hot regions to spread the load over the cluster), or go with larger region sizes if your cell sizes tend to be largish (100k and up).
In HBase 0.98, experimental stripe compactions feature was added that would allow for larger regions, especially for log data.
See <<ops.stripe,ops.stripe>>.
[[ops.capacity.regions.total]]
==== Total data size per region server
According to above numbers for region size and number of regions per region server, in an optimistic estimate 10 GB x 100 regions per RS will give up to 1TB served per region server, which is in line with some of the reported multi-PB use cases.
However, it is important to think about the data vs cache size ratio at the RS level.
With 1TB of data per server and 10 GB block cache, only 1% of the data will be cached, which may barely cover all block indices.
[[ops.capacity.config]]
=== Initial configuration and tuning
First, see <<important_configurations,important configurations>>.
Note that some configurations, more than others, depend on specific scenarios.
Pay special attention to:
* <<hbase.regionserver.handler.count,hbase.regionserver.handler.count>> - request handler thread count, vital for high-throughput workloads.
* <<config.wals,config.wals>> - the blocking number of WAL files depends on your memstore configuration and should be set accordingly to prevent potential blocking when doing high volume of writes.
Then, there are some considerations when setting up your cluster and tables.
[[ops.capacity.config.compactions]]
==== Compactions
Depending on read/write volume and latency requirements, optimal compaction settings may be different.
See <<compaction,compaction>> for some details.
When provisioning for large data sizes, however, it's good to keep in mind that compactions can affect write throughput.
Thus, for write-intensive workloads, you may opt for less frequent compactions and more store files per regions.
Minimum number of files for compactions (`hbase.hstore.compaction.min`) can be set to higher value; <<hbase.hstore.blockingStoreFiles,hbase.hstore.blockingStoreFiles>> should also be increased, as more files might accumulate in such case.
You may also consider manually managing compactions: <<managed.compactions,managed.compactions>>
[[ops.capacity.config.presplit]]
==== Pre-splitting the table
Based on the target number of the regions per RS (see <<ops.capacity.regions.count,ops.capacity.regions.count>>) and number of RSes, one can pre-split the table at creation time.
This would both avoid some costly splitting as the table starts to fill up, and ensure that the table starts out already distributed across many servers.
If the table is expected to grow large enough to justify that, at least one region per RS should be created.
It is not recommended to split immediately into the full target number of regions (e.g.
50 * number of RSes), but a low intermediate value can be chosen.
For multiple tables, it is recommended to be conservative with presplitting (e.g.
pre-split 1 region per RS at most), especially if you don't know how much each table will grow.
If you split too much, you may end up with too many regions, with some tables having too many small regions.
For pre-splitting howto, see <<manual_region_splitting_decisions,manual region splitting decisions>> and <<precreate.regions,precreate.regions>>.
[[table.rename]]
== 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.
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:
----
hbase shell> disable 'tableName'
hbase shell> snapshot 'tableName', 'tableSnapshot'
hbase shell> clone_snapshot 'tableSnapshot', 'newTableName'
hbase shell> delete_snapshot 'tableSnapshot'
hbase shell> drop 'tableName'
----
or in code it would be as follows:
[source,java]
----
void rename(Admin admin, String oldTableName, TableName newTableName) {
String snapshotName = randomName();
admin.disableTable(oldTableName);
admin.snapshot(snapshotName, oldTableName);
admin.cloneSnapshot(snapshotName, newTableName);
admin.deleteSnapshot(snapshotName);
admin.deleteTable(oldTableName);
}
----