4 * Licensed to the Apache Software Foundation (ASF) under one
5 * or more contributor license agreements. See the NOTICE file
6 * distributed with this work for additional information
7 * regarding copyright ownership. The ASF licenses this file
8 * to you under the Apache License, Version 2.0 (the
9 * "License"); you may not use this file except in compliance
10 * with the License. You may obtain a copy of the License at
12 * http://www.apache.org/licenses/LICENSE-2.0
14 * Unless required by applicable law or agreed to in writing, software
15 * distributed under the License is distributed on an "AS IS" BASIS,
16 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17 * See the License for the specific language governing permissions and
18 * limitations under the License.
29 :source-language: java
34 [[arch.overview.nosql]]
37 HBase is a type of "NoSQL" database.
38 "NoSQL" is a general term meaning that the database isn't an RDBMS which supports SQL as its primary access language, but there are many types of NoSQL databases: BerkeleyDB is an example of a local NoSQL database, whereas HBase is very much a distributed database.
39 Technically speaking, HBase is really more a "Data Store" than "Data Base" because it lacks many of the features you find in an RDBMS, such as typed columns, secondary indexes, triggers, and advanced query languages, etc.
41 However, HBase has many features which supports both linear and modular scaling.
42 HBase clusters expand by adding RegionServers that are hosted on commodity class servers.
43 If a cluster expands from 10 to 20 RegionServers, for example, it doubles both in terms of storage and as well as processing capacity.
44 An RDBMS can scale well, but only up to a point - specifically, the size of a single database
45 server - and for the best performance requires specialized hardware and storage devices.
46 HBase features of note are:
48 * Strongly consistent reads/writes: HBase is not an "eventually consistent" DataStore.
49 This makes it very suitable for tasks such as high-speed counter aggregation.
50 * Automatic sharding: HBase tables are distributed on the cluster via regions, and regions are automatically split and re-distributed as your data grows.
51 * Automatic RegionServer failover
52 * Hadoop/HDFS Integration: HBase supports HDFS out of the box as its distributed file system.
53 * MapReduce: HBase supports massively parallelized processing via MapReduce for using HBase as both source and sink.
54 * Java Client API: HBase supports an easy to use Java API for programmatic access.
55 * Thrift/REST API: HBase also supports Thrift and REST for non-Java front-ends.
56 * Block Cache and Bloom Filters: HBase supports a Block Cache and Bloom Filters for high volume query optimization.
57 * Operational Management: HBase provides build-in web-pages for operational insight as well as JMX metrics.
59 [[arch.overview.when]]
60 === When Should I Use HBase?
62 HBase isn't suitable for every problem.
64 First, make sure you have enough data.
65 If you have hundreds of millions or billions of rows, then HBase is a good candidate.
66 If you only have a few thousand/million rows, then using a traditional RDBMS might be a better choice due to the fact that all of your data might wind up on a single node (or two) and the rest of the cluster may be sitting idle.
68 Second, make sure you can live without all the extra features that an RDBMS provides (e.g., typed columns, secondary indexes, transactions, advanced query languages, etc.) An application built against an RDBMS cannot be "ported" to HBase by simply changing a JDBC driver, for example.
69 Consider moving from an RDBMS to HBase as a complete redesign as opposed to a port.
71 Third, make sure you have enough hardware.
72 Even HDFS doesn't do well with anything less than 5 DataNodes (due to things such as HDFS block replication which has a default of 3), plus a NameNode.
74 HBase can run quite well stand-alone on a laptop - but this should be considered a development configuration only.
76 [[arch.overview.hbasehdfs]]
77 === What Is The Difference Between HBase and Hadoop/HDFS?
79 link:https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html[HDFS] is a distributed file system that is well suited for the storage of large files.
80 Its documentation states that it is not, however, a general purpose file system, and does not provide fast individual record lookups in files.
81 HBase, on the other hand, is built on top of HDFS and provides fast record lookups (and updates) for large tables.
82 This can sometimes be a point of conceptual confusion.
83 HBase internally puts your data in indexed "StoreFiles" that exist on HDFS for high-speed lookups.
84 See the <<datamodel>> and the rest of this chapter for more information on how HBase achieves its goals.
89 The catalog table `hbase:meta` exists as an HBase table and is filtered out of the HBase shell's `list` command, but is in fact a table just like any other.
94 The `hbase:meta` table (previously called `.META.`) keeps a list of all regions in the system, and the location of `hbase:meta` is stored in ZooKeeper.
96 The `hbase:meta` table structure is as follows:
100 * Region key of the format (`[table],[region start key],[region id]`)
104 * `info:regioninfo` (serialized link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HRegionInfo.html[HRegionInfo] instance for this region)
105 * `info:server` (server:port of the RegionServer containing this region)
106 * `info:serverstartcode` (start-time of the RegionServer process containing this region)
108 When a table is in the process of splitting, two other columns will be created, called `info:splitA` and `info:splitB`.
109 These columns represent the two daughter regions.
110 The values for these columns are also serialized HRegionInfo instances.
111 After the region has been split, eventually this row will be deleted.
116 The empty key is used to denote table start and table end.
117 A region with an empty start key is the first region in a table.
118 If a region has both an empty start and an empty end key, it is the only region in the table
121 In the (hopefully unlikely) event that programmatic processing of catalog metadata
122 is required, see the link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/RegionInfo.html#parseFrom-byte:A-[RegionInfo.parseFrom] utility.
124 [[arch.catalog.startup]]
125 === Startup Sequencing
127 First, the location of `hbase:meta` is looked up in ZooKeeper.
128 Next, `hbase:meta` is updated with server and startcode values.
130 For information on region-RegionServer assignment, see <<regions.arch.assignment>>.
132 [[architecture.client]]
135 The HBase client finds the RegionServers that are serving the particular row range of interest.
136 It does this by querying the `hbase:meta` table.
137 See <<arch.catalog.meta>> for details.
138 After locating the required region(s), the client contacts the RegionServer serving that region, rather than going through the master, and issues the read or write request.
139 This information is cached in the client so that subsequent requests need not go through the lookup process.
140 Should a region be reassigned either by the master load balancer or because a RegionServer has died, the client will requery the catalog tables to determine the new location of the user region.
142 See <<master.runtime>> for more information about the impact of the Master on HBase Client communication.
144 Administrative functions are done via an instance of link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html[Admin]
146 [[client.connections]]
147 === Cluster Connections
149 The API changed in HBase 1.0. For connection configuration information, see <<client_dependencies>>.
151 ==== API as of HBase 1.0.0
153 It's been cleaned up and users are returned Interfaces to work against rather than particular types.
154 In HBase 1.0, obtain a `Connection` object from `ConnectionFactory` and thereafter, get from it instances of `Table`, `Admin`, and `RegionLocator` on an as-need basis.
155 When done, close the obtained instances.
156 Finally, be sure to cleanup your `Connection` instance before exiting.
157 `Connections` are heavyweight objects but thread-safe so you can create one for your application and keep the instance around.
158 `Table`, `Admin` and `RegionLocator` instances are lightweight.
159 Create as you go and then let go as soon as you are done by closing them.
160 See the link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/package-summary.html[Client Package Javadoc Description] for example usage of the new HBase 1.0 API.
162 ==== API before HBase 1.0.0
164 Instances of `HTable` are the way to interact with an HBase cluster earlier than 1.0.0. _link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table] instances are not thread-safe_. Only one thread can use an instance of Table at any given time.
165 When creating Table instances, it is advisable to use the same link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HBaseConfiguration[HBaseConfiguration] instance.
166 This will ensure sharing of ZooKeeper and socket instances to the RegionServers which is usually what you want.
167 For example, this is preferred:
171 HBaseConfiguration conf = HBaseConfiguration.create();
172 HTable table1 = new HTable(conf, "myTable");
173 HTable table2 = new HTable(conf, "myTable");
180 HBaseConfiguration conf1 = HBaseConfiguration.create();
181 HTable table1 = new HTable(conf1, "myTable");
182 HBaseConfiguration conf2 = HBaseConfiguration.create();
183 HTable table2 = new HTable(conf2, "myTable");
186 For more information about how connections are handled in the HBase client, see link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/ConnectionFactory.html[ConnectionFactory].
188 [[client.connection.pooling]]
189 ===== Connection Pooling
191 For applications which require high-end multithreaded access (e.g., web-servers or application servers that may serve many application threads in a single JVM), you can pre-create a `Connection`, as shown in the following example:
193 .Pre-Creating a `Connection`
197 // Create a connection to the cluster.
198 Configuration conf = HBaseConfiguration.create();
199 try (Connection connection = ConnectionFactory.createConnection(conf);
200 Table table = connection.getTable(TableName.valueOf(tablename))) {
201 // use table as needed, the table returned is lightweight
206 .`HTablePool` is Deprecated
209 Previous versions of this guide discussed `HTablePool`, which was deprecated in HBase 0.94, 0.95, and 0.96, and removed in 0.98.1, by link:https://issues.apache.org/jira/browse/HBASE-6580[HBASE-6580], or `HConnection`, which is deprecated in HBase 1.0 by `Connection`.
210 Please use link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Connection.html[Connection] instead.
213 [[client.writebuffer]]
214 === WriteBuffer and Batch Methods
216 In HBase 1.0 and later, link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTable.html[HTable] is deprecated in favor of link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table]. `Table` does not use autoflush. To do buffered writes, use the BufferedMutator class.
218 In HBase 2.0 and later, link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTable.html[HTable] does not use BufferedMutator to execute the ``Put`` operation. Refer to link:https://issues.apache.org/jira/browse/HBASE-18500[HBASE-18500] for more information.
220 For additional information on write durability, review the link:/acid-semantics.html[ACID semantics] page.
222 For fine-grained control of batching of ``Put``s or ``Delete``s, see the link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#batch-java.util.List-java.lang.Object:A-[batch] methods on Table.
225 === Asynchronous Client ===
227 It is a new API introduced in HBase 2.0 which aims to provide the ability to access HBase asynchronously.
229 You can obtain an `AsyncConnection` from `ConnectionFactory`, and then get a asynchronous table instance from it to access HBase. When done, close the `AsyncConnection` instance(usually when your program exits).
231 For the asynchronous table, most methods have the same meaning with the old `Table` interface, expect that the return value is wrapped with a CompletableFuture usually. We do not have any buffer here so there is no close method for asynchronous table, you do not need to close it. And it is thread safe.
233 There are several differences for scan:
235 * There is still a `getScanner` method which returns a `ResultScanner`. You can use it in the old way and it works like the old `ClientAsyncPrefetchScanner`.
236 * There is a `scanAll` method which will return all the results at once. It aims to provide a simpler way for small scans which you want to get the whole results at once usually.
237 * The Observer Pattern. There is a scan method which accepts a `ScanResultConsumer` as a parameter. It will pass the results to the consumer.
239 Notice that `AsyncTable` interface is templatized. The template parameter specifies the type of `ScanResultConsumerBase` used by scans, which means the observer style scan APIs are different. The two types of scan consumers are - `ScanResultConsumer` and `AdvancedScanResultConsumer`.
241 `ScanResultConsumer` needs a separate thread pool which is used to execute the callbacks registered to the returned CompletableFuture. Because the use of separate thread pool frees up RPC threads, callbacks are free to do anything. Use this if the callbacks are not quick, or when in doubt.
243 `AdvancedScanResultConsumer` executes callbacks inside the framework thread. It is not allowed to do time consuming work in the callbacks else it will likely block the framework threads and cause very bad performance impact. As its name, it is designed for advanced users who want to write high performance code. See `org.apache.hadoop.hbase.client.example.HttpProxyExample` for how to write fully asynchronous code with it.
246 === Asynchronous Admin ===
248 You can obtain an `AsyncConnection` from `ConnectionFactory`, and then get a `AsyncAdmin` instance from it to access HBase. Notice that there are two `getAdmin` methods to get a `AsyncAdmin` instance. One method has one extra thread pool parameter which is used to execute callbacks. It is designed for normal users. Another method doesn't need a thread pool and all the callbacks are executed inside the framework thread so it is not allowed to do time consuming works in the callbacks. It is designed for advanced users.
250 The default `getAdmin` methods will return a `AsyncAdmin` instance which use default configs. If you want to customize some configs, you can use `getAdminBuilder` methods to get a `AsyncAdminBuilder` for creating `AsyncAdmin` instance. Users are free to only set the configs they care about to create a new `AsyncAdmin` instance.
252 For the `AsyncAdmin` interface, most methods have the same meaning with the old `Admin` interface, expect that the return value is wrapped with a CompletableFuture usually.
254 For most admin operations, when the returned CompletableFuture is done, it means the admin operation has also been done. But for compact operation, it only means the compact request was sent to HBase and may need some time to finish the compact operation. For `rollWALWriter` method, it only means the rollWALWriter request was sent to the region server and may need some time to finish the `rollWALWriter` operation.
256 For region name, we only accept `byte[]` as the parameter type and it may be a full region name or a encoded region name. For server name, we only accept `ServerName` as the parameter type. For table name, we only accept `TableName` as the parameter type. For `list*` operations, we only accept `Pattern` as the parameter type if you want to do regex matching.
261 Information on non-Java clients and custom protocols is covered in <<external_apis>>
264 == Client Request Filters
266 link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html[Get] and link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html[Scan] instances can be optionally configured with link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/Filter.html[filters] which are applied on the RegionServer.
268 Filters can be confusing because there are many different types, and it is best to approach them by understanding the groups of Filter functionality.
270 [[client.filter.structural]]
273 Structural Filters contain other Filters.
275 [[client.filter.structural.fl]]
278 link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FilterList.html[FilterList] represents a list of Filters with a relationship of `FilterList.Operator.MUST_PASS_ALL` or `FilterList.Operator.MUST_PASS_ONE` between the Filters.
279 The following example shows an 'or' between two Filters (checking for either 'my value' or 'my other value' on the same attribute).
283 FilterList list = new FilterList(FilterList.Operator.MUST_PASS_ONE);
284 SingleColumnValueFilter filter1 = new SingleColumnValueFilter(
287 CompareOperator.EQUAL,
288 Bytes.toBytes("my value")
291 SingleColumnValueFilter filter2 = new SingleColumnValueFilter(
294 CompareOperator.EQUAL,
295 Bytes.toBytes("my other value")
298 scan.setFilter(list);
304 [[client.filter.cv.scvf]]
305 ==== SingleColumnValueFilter
307 A SingleColumnValueFilter (see:
308 https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.html)
309 can be used to test column values for equivalence (`CompareOperaor.EQUAL`),
310 inequality (`CompareOperaor.NOT_EQUAL`), or ranges (e.g., `CompareOperaor.GREATER`). The following is an
311 example of testing equivalence of a column to a String value "my value"...
315 SingleColumnValueFilter filter = new SingleColumnValueFilter(
318 CompareOperaor.EQUAL,
319 Bytes.toBytes("my value")
321 scan.setFilter(filter);
324 [[client.filter.cv.cvf]]
325 ==== ColumnValueFilter
327 Introduced in HBase-2.0.0 version as a complementation of SingleColumnValueFilter, ColumnValueFilter
328 gets matched cell only, while SingleColumnValueFilter gets the entire row
329 (has other columns and values) to which the matched cell belongs. Parameters of constructor of
330 ColumnValueFilter are the same as SingleColumnValueFilter.
333 ColumnValueFilter filter = new ColumnValueFilter(
336 CompareOperaor.EQUAL,
337 Bytes.toBytes("my value")
339 scan.setFilter(filter);
342 Note. For simple query like "equals to a family:qualifier:value", we highly recommend to use the
343 following way instead of using SingleColumnValueFilter or ColumnValueFilter:
346 Scan scan = new Scan();
347 scan.addColumn(Bytes.toBytes("family"), Bytes.toBytes("qualifier"));
348 ValueFilter vf = new ValueFilter(CompareOperator.EQUAL,
349 new BinaryComparator(Bytes.toBytes("value")));
353 This scan will restrict to the specified column 'family:qualifier', avoiding scan unrelated
354 families and columns, which has better performance, and `ValueFilter` is the condition used to do
357 But if query is much more complicated beyond this book, then please make your good choice case by case.
359 [[client.filter.cvp]]
360 === Column Value Comparators
362 There are several Comparator classes in the Filter package that deserve special mention.
363 These Comparators are used in concert with other Filters, such as <<client.filter.cv.scvf>>.
365 [[client.filter.cvp.rcs]]
366 ==== RegexStringComparator
368 link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/RegexStringComparator.html[RegexStringComparator] supports regular expressions for value comparisons.
372 RegexStringComparator comp = new RegexStringComparator("my."); // any value that starts with 'my'
373 SingleColumnValueFilter filter = new SingleColumnValueFilter(
376 CompareOperaor.EQUAL,
379 scan.setFilter(filter);
382 See the Oracle JavaDoc for link:http://download.oracle.com/javase/6/docs/api/java/util/regex/Pattern.html[supported RegEx patterns in Java].
384 [[client.filter.cvp.substringcomparator]]
385 ==== SubstringComparator
387 link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/SubstringComparator.html[SubstringComparator] can be used to determine if a given substring exists in a value.
388 The comparison is case-insensitive.
393 SubstringComparator comp = new SubstringComparator("y val"); // looking for 'my value'
394 SingleColumnValueFilter filter = new SingleColumnValueFilter(
397 CompareOperaor.EQUAL,
400 scan.setFilter(filter);
403 [[client.filter.cvp.bfp]]
404 ==== BinaryPrefixComparator
406 See link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.html[BinaryPrefixComparator].
408 [[client.filter.cvp.bc]]
409 ==== BinaryComparator
411 See link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/BinaryComparator.html[BinaryComparator].
413 [[client.filter.kvm]]
414 === KeyValue Metadata
416 As HBase stores data internally as KeyValue pairs, KeyValue Metadata Filters evaluate the existence of keys (i.e., ColumnFamily:Column qualifiers) for a row, as opposed to values the previous section.
418 [[client.filter.kvm.ff]]
421 link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FamilyFilter.html[FamilyFilter] can be used to filter on the ColumnFamily.
422 It is generally a better idea to select ColumnFamilies in the Scan than to do it with a Filter.
424 [[client.filter.kvm.qf]]
427 link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/QualifierFilter.html[QualifierFilter] can be used to filter based on Column (aka Qualifier) name.
429 [[client.filter.kvm.cpf]]
430 ==== ColumnPrefixFilter
432 link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.html[ColumnPrefixFilter] can be used to filter based on the lead portion of Column (aka Qualifier) names.
434 A ColumnPrefixFilter seeks ahead to the first column matching the prefix in each row and for each involved column family.
435 It can be used to efficiently get a subset of the columns in very wide rows.
437 Note: The same column qualifier can be used in different column families.
438 This filter returns all matching columns.
440 Example: Find all columns in a row and family that start with "abc"
447 byte[] prefix = Bytes.toBytes("abc");
448 Scan scan = new Scan(row, row); // (optional) limit to one row
449 scan.addFamily(family); // (optional) limit to one family
450 Filter f = new ColumnPrefixFilter(prefix);
452 scan.setBatch(10); // set this if there could be many columns returned
453 ResultScanner rs = t.getScanner(scan);
454 for (Result r = rs.next(); r != null; r = rs.next()) {
455 for (KeyValue kv : r.raw()) {
456 // each kv represents a column
462 [[client.filter.kvm.mcpf]]
463 ==== MultipleColumnPrefixFilter
465 link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.html[MultipleColumnPrefixFilter] behaves like ColumnPrefixFilter but allows specifying multiple prefixes.
467 Like ColumnPrefixFilter, MultipleColumnPrefixFilter efficiently seeks ahead to the first column matching the lowest prefix and also seeks past ranges of columns between prefixes.
468 It can be used to efficiently get discontinuous sets of columns from very wide rows.
470 Example: Find all columns in a row and family that start with "abc" or "xyz"
477 byte[][] prefixes = new byte[][] {Bytes.toBytes("abc"), Bytes.toBytes("xyz")};
478 Scan scan = new Scan(row, row); // (optional) limit to one row
479 scan.addFamily(family); // (optional) limit to one family
480 Filter f = new MultipleColumnPrefixFilter(prefixes);
482 scan.setBatch(10); // set this if there could be many columns returned
483 ResultScanner rs = t.getScanner(scan);
484 for (Result r = rs.next(); r != null; r = rs.next()) {
485 for (KeyValue kv : r.raw()) {
486 // each kv represents a column
492 [[client.filter.kvm.crf]]
493 ==== ColumnRangeFilter
495 A link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/ColumnRangeFilter.html[ColumnRangeFilter] allows efficient intra row scanning.
497 A ColumnRangeFilter can seek ahead to the first matching column for each involved column family.
498 It can be used to efficiently get a 'slice' of the columns of a very wide row.
500 you have a million columns in a row but you only want to look at columns bbbb-bbdd.
502 Note: The same column qualifier can be used in different column families.
503 This filter returns all matching columns.
505 Example: Find all columns in a row and family between "bbbb" (inclusive) and "bbdd" (inclusive)
512 byte[] startColumn = Bytes.toBytes("bbbb");
513 byte[] endColumn = Bytes.toBytes("bbdd");
514 Scan scan = new Scan(row, row); // (optional) limit to one row
515 scan.addFamily(family); // (optional) limit to one family
516 Filter f = new ColumnRangeFilter(startColumn, true, endColumn, true);
518 scan.setBatch(10); // set this if there could be many columns returned
519 ResultScanner rs = t.getScanner(scan);
520 for (Result r = rs.next(); r != null; r = rs.next()) {
521 for (KeyValue kv : r.raw()) {
522 // each kv represents a column
528 Note: Introduced in HBase 0.92
530 [[client.filter.row]]
533 [[client.filter.row.rf]]
536 It is generally a better idea to use the startRow/stopRow methods on Scan for row selection, however link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/RowFilter.html[RowFilter] can also be used.
538 [[client.filter.utility]]
541 [[client.filter.utility.fkof]]
542 ==== FirstKeyOnlyFilter
544 This is primarily used for rowcount jobs.
545 See link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.html[FirstKeyOnlyFilter].
547 [[architecture.master]]
550 `HMaster` is the implementation of the Master Server.
551 The Master server is responsible for monitoring all RegionServer instances in the cluster, and is the interface for all metadata changes.
552 In a distributed cluster, the Master typically runs on the <<arch.hdfs.nn>>.
553 J Mohamed Zahoor goes into some more detail on the Master Architecture in this blog posting, link:http://blog.zahoor.in/2012/08/hbase-hmaster-architecture/[HBase HMaster Architecture ].
558 If run in a multi-Master environment, all Masters compete to run the cluster.
559 If the active Master loses its lease in ZooKeeper (or the Master shuts down), then the remaining Masters jostle to take over the Master role.
564 A common dist-list question involves what happens to an HBase cluster when the Master goes down.
565 Because the HBase client talks directly to the RegionServers, the cluster can still function in a "steady state". Additionally, per <<arch.catalog>>, `hbase:meta` exists as an HBase table and is not resident in the Master.
566 However, the Master controls critical functions such as RegionServer failover and completing region splits.
567 So while the cluster can still run for a short time without the Master, the Master should be restarted as soon as possible.
572 The methods exposed by `HMasterInterface` are primarily metadata-oriented methods:
574 * Table (createTable, modifyTable, removeTable, enable, disable)
575 * ColumnFamily (addColumn, modifyColumn, removeColumn)
576 * Region (move, assign, unassign) For example, when the `Admin` method `disableTable` is invoked, it is serviced by the Master server.
581 The Master runs several background threads:
583 [[master.processes.loadbalancer]]
586 Periodically, and when there are no regions in transition, a load balancer will run and move regions around to balance the cluster's load.
587 See <<balancer_config>> for configuring this property.
589 See <<regions.arch.assignment>> for more information on region assignment.
591 [[master.processes.catalog]]
594 Periodically checks and cleans up the `hbase:meta` table.
595 See <<arch.catalog.meta>> for more information on the meta table.
600 HMaster records administrative operations and their running states, such as the handling of a crashed server,
601 table creation, and other DDLs, into its own WAL file. The WALs are stored under the MasterProcWALs
602 directory. The Master WALs are not like RegionServer WALs. Keeping up the Master WAL allows
603 us run a state machine that is resilient across Master failures. For example, if a HMaster was in the
604 middle of creating a table encounters an issue and fails, the next active HMaster can take up where
605 the previous left off and carry the operation to completion. Since hbase-2.0.0, a
606 new AssignmentManager (A.K.A AMv2) was introduced and the HMaster handles region assignment
607 operations, server crash processing, balancing, etc., all via AMv2 persisting all state and
608 transitions into MasterProcWALs rather than up into ZooKeeper, as we do in hbase-1.x.
610 See <<amv2>> (and <<pv2>> for its basis) if you would like to learn more about the new
614 ==== Configurations for MasterProcWAL
615 Here are the list of configurations that effect MasterProcWAL operation.
616 You should not have to change your defaults.
618 [[hbase.procedure.store.wal.periodic.roll.msec]]
619 *`hbase.procedure.store.wal.periodic.roll.msec`*::
622 Frequency of generating a new WAL
625 `1h (3600000 in msec)`
627 [[hbase.procedure.store.wal.roll.threshold]]
628 *`hbase.procedure.store.wal.roll.threshold`*::
631 Threshold in size before the WAL rolls. Every time the WAL reaches this size or the above period, 1 hour, passes since last log roll, the HMaster will generate a new WAL.
634 `32MB (33554432 in byte)`
636 [[hbase.procedure.store.wal.warn.threshold]]
637 *`hbase.procedure.store.wal.warn.threshold`*::
640 If the number of WALs goes beyond this threshold, the following message should appear in the HMaster log with WARN level when rolling.
642 procedure WALs count=xx above the warning threshold 64. check running procedures to see if something is stuck.
648 [[hbase.procedure.store.wal.max.retries.before.roll]]
649 *`hbase.procedure.store.wal.max.retries.before.roll`*::
652 Max number of retry when syncing slots (records) to its underlying storage, such as HDFS. Every attempt, the following message should appear in the HMaster log.
654 unable to sync slots, retry=xx
660 [[hbase.procedure.store.wal.sync.failure.roll.max]]
661 *`hbase.procedure.store.wal.sync.failure.roll.max`*::
664 After the above 3 retrials, the log is rolled and the retry count is reset to 0, thereon a new set of retrial starts. This configuration controls the max number of attempts of log rolling upon sync failure. That is, HMaster is allowed to fail to sync 9 times in total. Once it exceeds, the following log should appear in the HMaster log.
666 Sync slots after log roll failed, abort.
671 [[regionserver.arch]]
674 `HRegionServer` is the RegionServer implementation.
675 It is responsible for serving and managing regions.
676 In a distributed cluster, a RegionServer runs on a <<arch.hdfs.dn>>.
678 [[regionserver.arch.api]]
681 The methods exposed by `HRegionRegionInterface` contain both data-oriented and region-maintenance methods:
683 * Data (get, put, delete, next, etc.)
684 * Region (splitRegion, compactRegion, etc.) For example, when the `Admin` method `majorCompact` is invoked on a table, the client is actually iterating through all regions for the specified table and requesting a major compaction directly to each region.
686 [[regionserver.arch.processes]]
689 The RegionServer runs a variety of background threads:
691 [[regionserver.arch.processes.compactsplit]]
692 ==== CompactSplitThread
694 Checks for splits and handle minor compactions.
696 [[regionserver.arch.processes.majorcompact]]
697 ==== MajorCompactionChecker
699 Checks for major compactions.
701 [[regionserver.arch.processes.memstore]]
704 Periodically flushes in-memory writes in the MemStore to StoreFiles.
706 [[regionserver.arch.processes.log]]
709 Periodically checks the RegionServer's WAL.
713 Coprocessors were added in 0.92.
714 There is a thorough link:https://blogs.apache.org/hbase/entry/coprocessor_introduction[Blog Overview of CoProcessors] posted.
715 Documentation will eventually move to this reference guide, but the blog is the most current information available at this time.
720 HBase provides two different BlockCache implementations to cache data read from HDFS:
721 the default on-heap `LruBlockCache` and the `BucketCache`, which is (usually) off-heap.
722 This section discusses benefits and drawbacks of each implementation, how to choose the
723 appropriate option, and configuration options for each.
725 .Block Cache Reporting: UI
728 See the RegionServer UI for detail on caching deploy.
729 See configurations, sizings, current usage, time-in-the-cache, and even detail on block counts and types.
734 `LruBlockCache` is the original implementation, and is entirely within the Java heap.
735 `BucketCache` is optional and mainly intended for keeping block cache data off-heap, although `BucketCache` can also be a file-backed cache.
737 When you enable BucketCache, you are enabling a two tier caching system. We used to describe the
738 tiers as "L1" and "L2" but have deprecated this terminology as of hbase-2.0.0. The "L1" cache referred to an
739 instance of LruBlockCache and "L2" to an off-heap BucketCache. Instead, when BucketCache is enabled,
740 all DATA blocks are kept in the BucketCache tier and meta blocks -- INDEX and BLOOM blocks -- are on-heap in the `LruBlockCache`.
741 Management of these two tiers and the policy that dictates how blocks move between them is done by `CombinedBlockCache`.
743 [[cache.configurations]]
744 ==== General Cache Configurations
746 Apart from the cache implementation itself, you can set some general configuration options to control how the cache performs.
747 See link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html[CacheConfig].
748 After setting any of these options, restart or rolling restart your cluster for the configuration to take effect.
749 Check logs for errors or unexpected behavior.
751 See also <<blockcache.prefetch>>, which discusses a new option introduced in link:https://issues.apache.org/jira/browse/HBASE-9857[HBASE-9857].
753 [[block.cache.design]]
754 ==== LruBlockCache Design
756 The LruBlockCache is an LRU cache that contains three levels of block priority to allow for scan-resistance and in-memory ColumnFamilies:
758 * Single access priority: The first time a block is loaded from HDFS it normally has this priority and it will be part of the first group to be considered during evictions.
759 The advantage is that scanned blocks are more likely to get evicted than blocks that are getting more usage.
760 * Multi access priority: If a block in the previous priority group is accessed again, it upgrades to this priority.
761 It is thus part of the second group considered during evictions.
762 * In-memory access priority: If the block's family was configured to be "in-memory", it will be part of this priority disregarding the number of times it was accessed.
763 Catalog tables are configured like this.
764 This group is the last one considered during evictions.
766 To mark a column family as in-memory, call
770 HColumnDescriptor.setInMemory(true);
773 if creating a table from java, or set `IN_MEMORY => true` when creating or altering a table in the shell: e.g.
777 hbase(main):003:0> create 't', {NAME => 'f', IN_MEMORY => 'true'}
780 For more information, see the LruBlockCache source
782 [[block.cache.usage]]
783 ==== LruBlockCache Usage
785 Block caching is enabled by default for all the user tables which means that any read operation will load the LRU cache.
786 This might be good for a large number of use cases, but further tunings are usually required in order to achieve better performance.
787 An important concept is the link:http://en.wikipedia.org/wiki/Working_set_size[working set size], or WSS, which is: "the amount of memory needed to compute the answer to a problem". For a website, this would be the data that's needed to answer the queries over a short amount of time.
789 The way to calculate how much memory is available in HBase for caching is:
793 number of region servers * heap size * hfile.block.cache.size * 0.99
796 The default value for the block cache is 0.4 which represents 40% of the available heap.
797 The last value (99%) is the default acceptable loading factor in the LRU cache after which eviction is started.
798 The reason it is included in this equation is that it would be unrealistic to say that it is possible to use 100% of the available memory since this would make the process blocking from the point where it loads new blocks.
799 Here are some examples:
801 * One region server with the heap size set to 1 GB and the default block cache size will have 405 MB of block cache available.
802 * 20 region servers with the heap size set to 8 GB and a default block cache size will have 63.3 of block cache.
803 * 100 region servers with the heap size set to 24 GB and a block cache size of 0.5 will have about 1.16 TB of block cache.
805 Your data is not the only resident of the block cache.
806 Here are others that you may have to take into account:
809 The `hbase:meta` table is forced into the block cache and have the in-memory priority which means that they are harder to evict.
811 NOTE: The hbase:meta tables can occupy a few MBs depending on the number of regions.
814 An _HFile_ is the file format that HBase uses to store data in HDFS.
815 It contains a multi-layered index which allows HBase to seek to the data without having to read the whole file.
816 The size of those indexes is a factor of the block size (64KB by default), the size of your keys and the amount of data you are storing.
817 For big data sets it's not unusual to see numbers around 1GB per region server, although not all of it will be in cache because the LRU will evict indexes that aren't used.
820 The values that are stored are only half the picture, since each value is stored along with its keys (row key, family qualifier, and timestamp). See <<keysize>>.
823 Just like the HFile indexes, those data structures (when enabled) are stored in the LRU.
825 Currently the recommended way to measure HFile indexes and bloom filters sizes is to look at the region server web UI and checkout the relevant metrics.
826 For keys, sampling can be done by using the HFile command line tool and look for the average key size metric.
827 Since HBase 0.98.3, you can view details on BlockCache stats and metrics in a special Block Cache section in the UI.
829 It's generally bad to use block caching when the WSS doesn't fit in memory.
830 This is the case when you have for example 40GB available across all your region servers' block caches but you need to process 1TB of data.
831 One of the reasons is that the churn generated by the evictions will trigger more garbage collections unnecessarily.
832 Here are two use cases:
834 * Fully random reading pattern: This is a case where you almost never access the same row twice within a short amount of time such that the chance of hitting a cached block is close to 0.
835 Setting block caching on such a table is a waste of memory and CPU cycles, more so that it will generate more garbage to pick up by the JVM.
836 For more information on monitoring GC, see <<trouble.log.gc>>.
837 * Mapping a table: In a typical MapReduce job that takes a table in input, every row will be read only once so there's no need to put them into the block cache.
838 The Scan object has the option of turning this off via the setCaching method (set it to false). You can still keep block caching turned on on this table if you need fast random read access.
839 An example would be counting the number of rows in a table that serves live traffic, caching every block of that table would create massive churn and would surely evict data that's currently in use.
841 [[data.blocks.in.fscache]]
842 ===== Caching META blocks only (DATA blocks in fscache)
844 An interesting setup is one where we cache META blocks only and we read DATA blocks in on each access.
845 If the DATA blocks fit inside fscache, this alternative may make sense when access is completely random across a very large dataset.
846 To enable this setup, alter your table and for each column family set `BLOCKCACHE => 'false'`.
847 You are 'disabling' the BlockCache for this column family only. You can never disable the caching of META blocks.
848 Since link:https://issues.apache.org/jira/browse/HBASE-4683[HBASE-4683 Always cache index and bloom blocks], we will cache META blocks even if the BlockCache is disabled.
850 [[offheap.blockcache]]
851 ==== Off-heap Block Cache
853 [[enable.bucketcache]]
854 ===== How to Enable BucketCache
856 The usual deploy of BucketCache is via a managing class that sets up two caching tiers:
857 an on-heap cache implemented by LruBlockCache and a second cache implemented with BucketCache.
858 The managing class is link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.html[CombinedBlockCache] by default.
859 The previous link describes the caching 'policy' implemented by CombinedBlockCache.
860 In short, it works by keeping meta blocks -- INDEX and BLOOM in the on-heap LruBlockCache tier -- and DATA blocks are kept in the BucketCache tier.
863 Pre-hbase-2.0.0 versions::
864 Fetching will always be slower when fetching from BucketCache in pre-hbase-2.0.0,
865 as compared to the native on-heap LruBlockCache. However, latencies tend to be less
866 erratic across time, because there is less garbage collection when you use BucketCache since it is managing BlockCache allocations, not the GC.
867 If the BucketCache is deployed in off-heap mode, this memory is not managed by the GC at all.
868 This is why you'd use BucketCache in pre-2.0.0, so your latencies are less erratic,
869 to mitigate GCs and heap fragmentation, and so you can safely use more memory.
870 See Nick Dimiduk's link:http://www.n10k.com/blog/blockcache-101/[BlockCache 101] for comparisons running on-heap vs off-heap tests.
871 Also see link:https://people.apache.org/~stack/bc/[Comparing BlockCache Deploys] which finds that if your dataset fits inside your LruBlockCache deploy, use it otherwise if you are experiencing cache churn (or you want your cache to exist beyond the vagaries of java GC), use BucketCache.
874 one can configure the BucketCache so it receives the `victim` of an LruBlockCache eviction.
875 All Data and index blocks are cached in L1 first. When eviction happens from L1, the blocks (or `victims`) will get moved to L2.
876 Set `cacheDataInL1` via `(HColumnDescriptor.setCacheDataInL1(true)` or in the shell, creating or amending column families setting `CACHE_DATA_IN_L1` to true: e.g.
879 hbase(main):003:0> create 't', {NAME => 't', CONFIGURATION => {CACHE_DATA_IN_L1 => 'true'}}
882 hbase-2.0.0+ versions::
883 HBASE-11425 changed the HBase read path so it could hold the read-data off-heap avoiding copying of cached data on to the java heap.
884 See <<regionserver.offheap.readpath>>. In hbase-2.0.0, off-heap latencies approach those of on-heap cache latencies with the added
885 benefit of NOT provoking GC.
887 From HBase 2.0.0 onwards, the notions of L1 and L2 have been deprecated. When BucketCache is turned on, the DATA blocks will always go to BucketCache and INDEX/BLOOM blocks go to on heap LRUBlockCache. `cacheDataInL1` support hase been removed.
890 The BucketCache Block Cache can be deployed _off-heap_, _file_ or _mmaped_ file mode.
893 You set which via the `hbase.bucketcache.ioengine` setting.
894 Setting it to `offheap` will have BucketCache make its allocations off-heap, and an ioengine setting of `file:PATH_TO_FILE` will direct BucketCache to use file caching (Useful in particular if you have some fast I/O attached to the box such as SSDs). From 2.0.0, it is possible to have more than one file backing the BucketCache. This is very useful specially when the Cache size requirement is high. For multiple backing files, configure ioengine as `files:PATH_TO_FILE1,PATH_TO_FILE2,PATH_TO_FILE3`. BucketCache can be configured to use an mmapped file also. Configure ioengine as `mmap:PATH_TO_FILE` for this.
896 It is possible to deploy a tiered setup where we bypass the CombinedBlockCache policy and have BucketCache working as a strict L2 cache to the L1 LruBlockCache.
897 For such a setup, set `hbase.bucketcache.combinedcache.enabled` to `false`.
898 In this mode, on eviction from L1, blocks go to L2.
899 When a block is cached, it is cached first in L1.
900 When we go to look for a cached block, we look first in L1 and if none found, then search L2.
901 Let us call this deploy format, _Raw L1+L2_.
902 NOTE: This L1+L2 mode is removed from 2.0.0. When BucketCache is used, it will be strictly the DATA cache and the LruBlockCache will cache INDEX/META blocks.
904 Other BucketCache configs include: specifying a location to persist cache to across restarts, how many threads to use writing the cache, etc.
905 See the link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html[CacheConfig.html] class for configuration options and descriptions.
907 To check it enabled, look for the log line describing cache setup; it will detail how BucketCache has been deployed.
908 Also see the UI. It will detail the cache tiering and their configuration.
910 ====== BucketCache Example Configuration
911 This sample provides a configuration for a 4 GB off-heap BucketCache with a 1 GB on-heap cache.
913 Configuration is performed on the RegionServer.
915 Setting `hbase.bucketcache.ioengine` and `hbase.bucketcache.size` > 0 enables `CombinedBlockCache`.
916 Let us presume that the RegionServer has been set to run with a 5G heap: i.e. `HBASE_HEAPSIZE=5g`.
919 . First, edit the RegionServer's _hbase-env.sh_ and set `HBASE_OFFHEAPSIZE` to a value greater than the off-heap size wanted, in this case, 4 GB (expressed as 4G). Let's set it to 5G.
920 That'll be 4G for our off-heap cache and 1G for any other uses of off-heap memory (there are other users of off-heap memory other than BlockCache; e.g.
921 DFSClient in RegionServer can make use of off-heap memory). See <<direct.memory>>.
928 . Next, add the following configuration to the RegionServer's _hbase-site.xml_.
933 <name>hbase.bucketcache.ioengine</name>
934 <value>offheap</value>
937 <name>hfile.block.cache.size</name>
941 <name>hbase.bucketcache.size</name>
946 . Restart or rolling restart your cluster, and check the logs for any issues.
949 In the above, we set the BucketCache to be 4G.
950 We configured the on-heap LruBlockCache have 20% (0.2) of the RegionServer's heap size (0.2 * 5G = 1G). In other words, you configure the L1 LruBlockCache as you would normally (as if there were no L2 cache present).
952 link:https://issues.apache.org/jira/browse/HBASE-10641[HBASE-10641] introduced the ability to configure multiple sizes for the buckets of the BucketCache, in HBase 0.98 and newer.
953 To configurable multiple bucket sizes, configure the new property `hbase.bucketcache.bucket.sizes` to a comma-separated list of block sizes, ordered from smallest to largest, with no spaces.
954 The goal is to optimize the bucket sizes based on your data access patterns.
955 The following example configures buckets of size 4096 and 8192.
960 <name>hbase.bucketcache.bucket.sizes</name>
961 <value>4096,8192</value>
966 .Direct Memory Usage In HBase
969 The default maximum direct memory varies by JVM.
970 Traditionally it is 64M or some relation to allocated heap size (-Xmx) or no limit at all (JDK7 apparently). HBase servers use direct memory, in particular short-circuit reading (See <<perf.hdfs.configs.localread>>), the hosted DFSClient will allocate direct memory buffers. How much the DFSClient uses is not easy to quantify; it is the number of open HFiles * `hbase.dfs.client.read.shortcircuit.buffer.size` where `hbase.dfs.client.read.shortcircuit.buffer.size` is set to 128k in HBase -- see _hbase-default.xml_ default configurations.
971 If you do off-heap block caching, you'll be making use of direct memory.
972 The RPCServer uses a ByteBuffer pool. From 2.0.0, these buffers are off-heap ByteBuffers.
973 Starting your JVM, make sure the `-XX:MaxDirectMemorySize` setting in _conf/hbase-env.sh_ considers off-heap BlockCache (`hbase.bucketcache.size`), DFSClient usage, RPC side ByteBufferPool max size. This has to be bit higher than sum of off heap BlockCache size and max ByteBufferPool size. Allocating an extra of 1-2 GB for the max direct memory size has worked in tests. Direct memory, which is part of the Java process heap, is separate from the object heap allocated by -Xmx.
974 The value allocated by `MaxDirectMemorySize` must not exceed physical RAM, and is likely to be less than the total available RAM due to other memory requirements and system constraints.
976 You can see how much memory -- on-heap and off-heap/direct -- a RegionServer is configured to use and how much it is using at any one time by looking at the _Server Metrics: Memory_ tab in the UI.
977 It can also be gotten via JMX.
978 In particular the direct memory currently used by the server can be found on the `java.nio.type=BufferPool,name=direct` bean.
979 Terracotta has a link:http://terracotta.org/documentation/4.0/bigmemorygo/configuration/storage-options[good write up] on using off-heap memory in Java.
980 It is for their product BigMemory but a lot of the issues noted apply in general to any attempt at going off-heap. Check it out.
983 .hbase.bucketcache.percentage.in.combinedcache
986 This is a pre-HBase 1.0 configuration removed because it was confusing.
987 It was a float that you would set to some value between 0.0 and 1.0.
989 If the deploy was using CombinedBlockCache, then the LruBlockCache L1 size was calculated to be `(1 - hbase.bucketcache.percentage.in.combinedcache) * size-of-bucketcache` and the BucketCache size was `hbase.bucketcache.percentage.in.combinedcache * size-of-bucket-cache`.
990 where size-of-bucket-cache itself is EITHER the value of the configuration `hbase.bucketcache.size` IF it was specified as Megabytes OR `hbase.bucketcache.size` * `-XX:MaxDirectMemorySize` if `hbase.bucketcache.size` is between 0 and 1.0.
992 In 1.0, it should be more straight-forward.
993 Onheap LruBlockCache size is set as a fraction of java heap using `hfile.block.cache.size setting` (not the best name) and BucketCache is set as above in absolute Megabytes.
996 ==== Compressed BlockCache
998 link:https://issues.apache.org/jira/browse/HBASE-11331[HBASE-11331] introduced lazy BlockCache decompression, more simply referred to as compressed BlockCache.
999 When compressed BlockCache is enabled data and encoded data blocks are cached in the BlockCache in their on-disk format, rather than being decompressed and decrypted before caching.
1001 For a RegionServer hosting more data than can fit into cache, enabling this feature with SNAPPY compression has been shown to result in 50% increase in throughput and 30% improvement in mean latency while, increasing garbage collection by 80% and increasing overall CPU load by 2%. See HBASE-11331 for more details about how performance was measured and achieved.
1002 For a RegionServer hosting data that can comfortably fit into cache, or if your workload is sensitive to extra CPU or garbage-collection load, you may receive less benefit.
1004 The compressed BlockCache is disabled by default. To enable it, set `hbase.block.data.cachecompressed` to `true` in _hbase-site.xml_ on all RegionServers.
1006 [[regionserver.offheap]]
1007 === RegionServer Offheap Read/Write Path
1009 [[regionserver.offheap.readpath]]
1010 ==== Offheap read-path
1011 In hbase-2.0.0, link:https://issues.apache.org/jira/browse/HBASE-11425[HBASE-11425] changed the HBase read path so it
1012 could hold the read-data off-heap avoiding copying of cached data on to the java heap.
1013 This reduces GC pauses given there is less garbage made and so less to clear. The off-heap read path has a performance
1014 that is similar/better to that of the on-heap LRU cache. This feature is available since HBase 2.0.0.
1015 If the BucketCache is in `file` mode, fetching will always be slower compared to the native on-heap LruBlockCache.
1016 Refer to below blogs for more details and test results on off heaped read path
1017 link:https://blogs.apache.org/hbase/entry/offheaping_the_read_path_in[Offheaping the Read Path in Apache HBase: Part 1 of 2]
1018 and link:https://blogs.apache.org/hbase/entry/offheap-read-path-in-production[Offheap Read-Path in Production - The Alibaba story]
1020 For an end-to-end off-heaped read-path, first of all there should be an off-heap backed <<offheap.blockcache>>(BC). Configure 'hbase.bucketcache.ioengine' to off-heap in
1021 _hbase-site.xml_. Also specify the total capacity of the BC using `hbase.bucketcache.size` config. Please remember to adjust value of 'HBASE_OFFHEAPSIZE' in
1022 _hbase-env.sh_. This is how we specify the max possible off-heap memory allocation for the
1023 RegionServer java process. This should be bigger than the off-heap BC size. Please keep in mind that there is no default for `hbase.bucketcache.ioengine`
1024 which means the BC is turned OFF by default (See <<direct.memory>>).
1026 Next thing to tune is the ByteBuffer pool on the RPC server side.
1027 The buffers from this pool will be used to accumulate the cell bytes and create a result cell block to send back to the client side.
1028 `hbase.ipc.server.reservoir.enabled` can be used to turn this pool ON or OFF. By default this pool is ON and available. HBase will create off heap ByteBuffers
1029 and pool them. Please make sure not to turn this OFF if you want end-to-end off-heaping in read path.
1030 If this pool is turned off, the server will create temp buffers on heap to accumulate the cell bytes and make a result cell block. This can impact the GC on a highly read loaded server.
1031 The user can tune this pool with respect to how many buffers are in the pool and what should be the size of each ByteBuffer.
1032 Use the config `hbase.ipc.server.reservoir.initial.buffer.size` to tune each of the buffer sizes. Default is 64 KB.
1034 When the read pattern is a random row read load and each of the rows are smaller in size compared to this 64 KB, try reducing this.
1035 When the result size is larger than one ByteBuffer size, the server will try to grab more than one buffer and make a result cell block out of these. When the pool is running out of buffers, the server will end up creating temporary on-heap buffers.
1037 The maximum number of ByteBuffers in the pool can be tuned using the config 'hbase.ipc.server.reservoir.initial.max'. Its value defaults to 64 * region server handlers configured (See the config 'hbase.regionserver.handler.count'). The math is such that by default we consider 2 MB as the result cell block size per read result and each handler will be handling a read. For 2 MB size, we need 32 buffers each of size 64 KB (See default buffer size in pool). So per handler 32 ByteBuffers(BB). We allocate twice this size as the max BBs count such that one handler can be creating the response and handing it to the RPC Responder thread and then handling a new request creating a new response cell block (using pooled buffers). Even if the responder could not send back the first TCP reply immediately, our count should allow that we should still have enough buffers in our pool without having to make temporary buffers on the heap. Again for smaller sized random row reads, tune this max count. There are lazily created buffers and the count is the max count to be pooled.
1039 If you still see GC issues even after making end-to-end read path off-heap, look for issues in the appropriate buffer pool. Check the below RegionServer log with INFO level:
1042 Pool already reached its max capacity : XXX and no free buffers now. Consider increasing the value for 'hbase.ipc.server.reservoir.initial.max' ?
1045 The setting for _HBASE_OFFHEAPSIZE_ in _hbase-env.sh_ should consider this off heap buffer pool at the RPC side also. We need to config this max off heap size for the RegionServer as a bit higher than the sum of this max pool size and the off heap cache size. The TCP layer will also need to create direct bytebuffers for TCP communication. Also the DFS client will need some off-heap to do its workings especially if short-circuit reads are configured. Allocating an extra of 1 - 2 GB for the max direct memory size has worked in tests.
1047 If you are using co processors and refer the Cells in the read results, DO NOT store reference to these Cells out of the scope of the CP hook methods. Some times the CPs need store info about the cell (Like its row key) for considering in the next CP hook call etc. For such cases, pls clone the required fields of the entire Cell as per the use cases. [ See CellUtil#cloneXXX(Cell) APIs ]
1049 [[regionserver.offheap.writepath]]
1050 ==== Offheap write-path
1054 [[regionserver_splitting_implementation]]
1055 === RegionServer Splitting Implementation
1057 As write requests are handled by the region server, they accumulate in an in-memory storage system called the _memstore_. Once the memstore fills, its content are written to disk as additional store files. This event is called a _memstore flush_. As store files accumulate, the RegionServer will <<compaction,compact>> them into fewer, larger files. After each flush or compaction finishes, the amount of data stored in the region has changed. The RegionServer consults the region split policy to determine if the region has grown too large or should be split for another policy-specific reason. A region split request is enqueued if the policy recommends it.
1059 Logically, the process of splitting a region is simple. We find a suitable point in the keyspace of the region where we should divide the region in half, then split the region's data into two new regions at that point. The details of the process however are not simple. When a split happens, the newly created _daughter regions_ do not rewrite all the data into new files immediately. Instead, they create small files similar to symbolic link files, named link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/Reference.html[Reference files], which point to either the top or bottom part of the parent store file according to the split point. The reference file is used just like a regular data file, but only half of the records are considered. The region can only be split if there are no more references to the immutable data files of the parent region. Those reference files are cleaned gradually by compactions, so that the region will stop referring to its parents files, and can be split further.
1061 Although splitting the region is a local decision made by the RegionServer, the split process itself must coordinate with many actors. The RegionServer notifies the Master before and after the split, updates the `.META.` table so that clients can discover the new daughter regions, and rearranges the directory structure and data files in HDFS. Splitting is a multi-task process. To enable rollback in case of an error, the RegionServer keeps an in-memory journal about the execution state. The steps taken by the RegionServer to execute the split are illustrated in <<regionserver_split_process_image>>. Each step is labeled with its step number. Actions from RegionServers or Master are shown in red, while actions from the clients are show in green.
1063 [[regionserver_split_process_image]]
1064 .RegionServer Split Process
1065 image::region_split_process.png[Region Split Process]
1067 . The RegionServer decides locally to split the region, and prepares the split. *THE SPLIT TRANSACTION IS STARTED.* As a first step, the RegionServer acquires a shared read lock on the table to prevent schema modifications during the splitting process. Then it creates a znode in zookeeper under `/hbase/region-in-transition/region-name`, and sets the znode's state to `SPLITTING`.
1068 . The Master learns about this znode, since it has a watcher for the parent `region-in-transition` znode.
1069 . The RegionServer creates a sub-directory named `.splits` under the parent’s `region` directory in HDFS.
1070 . The RegionServer closes the parent region and marks the region as offline in its local data structures. *THE SPLITTING REGION IS NOW OFFLINE.* At this point, client requests coming to the parent region will throw `NotServingRegionException`. The client will retry with some backoff. The closing region is flushed.
1071 . The RegionServer creates region directories under the `.splits` directory, for daughter
1072 regions A and B, and creates necessary data structures. Then it splits the store files,
1073 in the sense that it creates two Reference files per store file in the parent region.
1074 Those reference files will point to the parent region's files.
1075 . The RegionServer creates the actual region directory in HDFS, and moves the reference files for each daughter.
1076 . The RegionServer sends a `Put` request to the `.META.` table, to set the parent as offline in the `.META.` table and add information about daughter regions. At this point, there won’t be individual entries in `.META.` for the daughters. Clients will see that the parent region is split if they scan `.META.`, but won’t know about the daughters until they appear in `.META.`. Also, if this `Put` to `.META`. succeeds, the parent will be effectively split. If the RegionServer fails before this RPC succeeds, Master and the next Region Server opening the region will clean dirty state about the region split. After the `.META.` update, though, the region split will be rolled-forward by Master.
1077 . The RegionServer opens daughters A and B in parallel.
1078 . The RegionServer adds the daughters A and B to `.META.`, together with information that it hosts the regions. *THE SPLIT REGIONS (DAUGHTERS WITH REFERENCES TO PARENT) ARE NOW ONLINE.* After this point, clients can discover the new regions and issue requests to them. Clients cache the `.META.` entries locally, but when they make requests to the RegionServer or `.META.`, their caches will be invalidated, and they will learn about the new regions from `.META.`.
1079 . The RegionServer updates znode `/hbase/region-in-transition/region-name` in ZooKeeper to state `SPLIT`, so that the master can learn about it. The balancer can freely re-assign the daughter regions to other region servers if necessary. *THE SPLIT TRANSACTION IS NOW FINISHED.*
1080 . After the split, `.META.` and HDFS will still contain references to the parent region. Those references will be removed when compactions in daughter regions rewrite the data files. Garbage collection tasks in the master periodically check whether the daughter regions still refer to the parent region's files. If not, the parent region will be removed.
1083 === Write Ahead Log (WAL)
1088 The _Write Ahead Log (WAL)_ records all changes to data in HBase, to file-based storage.
1089 Under normal operations, the WAL is not needed because data changes move from the MemStore to StoreFiles.
1090 However, if a RegionServer crashes or becomes unavailable before the MemStore is flushed, the WAL ensures that the changes to the data can be replayed.
1091 If writing to the WAL fails, the entire operation to modify the data fails.
1093 HBase uses an implementation of the link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/wal/WAL.html[WAL] interface.
1094 Usually, there is only one instance of a WAL per RegionServer. An exception
1095 is the RegionServer that is carrying _hbase:meta_; the _meta_ table gets its
1097 The RegionServer records Puts and Deletes to its WAL, before recording them
1098 these Mutations <<store.memstore>> for the affected <<store>>.
1103 Prior to 2.0, the interface for WALs in HBase was named `HLog`.
1104 In 0.94, HLog was the name of the implementation of the WAL.
1105 You will likely find references to the HLog in documentation tailored to these older versions.
1108 The WAL resides in HDFS in the _/hbase/WALs/_ directory, with subdirectories per region.
1110 For more general information about the concept of write ahead logs, see the Wikipedia
1111 link:http://en.wikipedia.org/wiki/Write-ahead_logging[Write-Ahead Log] article.
1116 In HBase, there are a number of WAL imlementations (or 'Providers'). Each is known
1117 by a short name label (that unfortunately is not always descriptive). You set the provider in
1118 _hbase-site.xml_ passing the WAL provder short-name as the value on the
1119 _hbase.wal.provider_ property (Set the provider for _hbase:meta_ using the
1120 _hbase.wal.meta_provider_ property, otherwise it uses the same provider configured
1121 by _hbase.wal.provider_).
1123 * _asyncfs_: The *default*. New since hbase-2.0.0 (HBASE-15536, HBASE-14790). This _AsyncFSWAL_ provider, as it identifies itself in RegionServer logs, is built on a new non-blocking dfsclient implementation. It is currently resident in the hbase codebase but intent is to move it back up into HDFS itself. WALs edits are written concurrently ("fan-out") style to each of the WAL-block replicas on each DataNode rather than in a chained pipeline as the default client does. Latencies should be better. See link:https://www.slideshare.net/HBaseCon/apache-hbase-improvements-and-practices-at-xiaomi[Apache HBase Improements and Practices at Xiaomi] at slide 14 onward for more detail on implementation.
1124 * _filesystem_: This was the default in hbase-1.x releases. It is built on the blocking _DFSClient_ and writes to replicas in classic _DFSCLient_ pipeline mode. In logs it identifies as _FSHLog_ or _FSHLogProvider_.
1125 * _multiwal_: This provider is made of multiple instances of _asyncfs_ or _filesystem_. See the next section for more on _multiwal_.
1127 Look for the lines like the below in the RegionServer log to see which provider is in place (The below shows the default AsyncFSWALProvider):
1130 2018-04-02 13:22:37,983 INFO [regionserver/ve0528:16020] wal.WALFactory: Instantiating WALProvider of type class org.apache.hadoop.hbase.wal.AsyncFSWALProvider
1133 NOTE: As the _AsyncFSWAL_ hacks into the internal of DFSClient implementation, it will be easily broken by upgrading the hadoop dependencies, even for a simple patch release. So if you do not specify the wal provider explicitly, we will first try to use the _asyncfs_, if failed, we will fall back to use _filesystem_. And notice that this may not always work, so if you still have problem starting HBase due to the problem of starting _AsyncFSWAL_, please specify _filesystem_ explicitly in the config file.
1135 NOTE: EC support has been added to hadoop-3.x, and it is incompatible with WAL as the EC output stream does not support hflush/hsync. In order to create a non-EC file in an EC directory, we need to use the new builder-based create API for _FileSystem_, but it is only introduced in hadoop-2.9+ and for HBase we still need to support hadoop-2.7.x. So please do not enable EC for the WAL directory until we find a way to deal with it.
1138 With a single WAL per RegionServer, the RegionServer must write to the WAL serially, because HDFS files must be sequential. This causes the WAL to be a performance bottleneck.
1140 HBase 1.0 introduces support MultiWal in link:https://issues.apache.org/jira/browse/HBASE-5699[HBASE-5699]. MultiWAL allows a RegionServer to write multiple WAL streams in parallel, by using multiple pipelines in the underlying HDFS instance, which increases total throughput during writes. This parallelization is done by partitioning incoming edits by their Region. Thus, the current implementation will not help with increasing the throughput to a single Region.
1142 RegionServers using the original WAL implementation and those using the MultiWAL implementation can each handle recovery of either set of WALs, so a zero-downtime configuration update is possible through a rolling restart.
1145 To configure MultiWAL for a RegionServer, set the value of the property `hbase.wal.provider` to `multiwal` by pasting in the following XML:
1150 <name>hbase.wal.provider</name>
1151 <value>multiwal</value>
1155 Restart the RegionServer for the changes to take effect.
1157 To disable MultiWAL for a RegionServer, unset the property and restart the RegionServer.
1167 A RegionServer serves many regions.
1168 All of the regions in a region server share the same active WAL file.
1169 Each edit in the WAL file includes information about which region it belongs to.
1170 When a region is opened, the edits in the WAL file which belong to that region need to be replayed.
1171 Therefore, edits in the WAL file must be grouped by region so that particular sets can be replayed to regenerate the data in a particular region.
1172 The process of grouping the WAL edits by region is called _log splitting_.
1173 It is a critical process for recovering data if a region server fails.
1175 Log splitting is done by the HMaster during cluster start-up or by the ServerShutdownHandler as a region server shuts down.
1176 So that consistency is guaranteed, affected regions are unavailable until data is restored.
1177 All WAL edits need to be recovered and replayed before a given region can become available again.
1178 As a result, regions affected by log splitting are unavailable until the process completes.
1180 .Procedure: Log Splitting, Step by Step
1181 . The _/hbase/WALs/<host>,<port>,<startcode>_ directory is renamed.
1183 Renaming the directory is important because a RegionServer may still be up and accepting requests even if the HMaster thinks it is down.
1184 If the RegionServer does not respond immediately and does not heartbeat its ZooKeeper session, the HMaster may interpret this as a RegionServer failure.
1185 Renaming the logs directory ensures that existing, valid WAL files which are still in use by an active but busy RegionServer are not written to by accident.
1187 The new directory is named according to the following pattern:
1190 /hbase/WALs/<host>,<port>,<startcode>-splitting
1193 An example of such a renamed directory might look like the following:
1196 /hbase/WALs/srv.example.com,60020,1254173957298-splitting
1199 . Each log file is split, one at a time.
1201 The log splitter reads the log file one edit entry at a time and puts each edit entry into the buffer corresponding to the edit's region.
1202 At the same time, the splitter starts several writer threads.
1203 Writer threads pick up a corresponding buffer and write the edit entries in the buffer to a temporary recovered edit file.
1204 The temporary edit file is stored to disk with the following naming pattern:
1207 /hbase/<table_name>/<region_id>/recovered.edits/.temp
1210 This file is used to store all the edits in the WAL log for this region.
1211 After log splitting completes, the _.temp_ file is renamed to the sequence ID of the first log written to the file.
1213 To determine whether all edits have been written, the sequence ID is compared to the sequence of the last edit that was written to the HFile.
1214 If the sequence of the last edit is greater than or equal to the sequence ID included in the file name, it is clear that all writes from the edit file have been completed.
1216 . After log splitting is complete, each affected region is assigned to a RegionServer.
1218 When the region is opened, the _recovered.edits_ folder is checked for recovered edits files.
1219 If any such files are present, they are replayed by reading the edits and saving them to the MemStore.
1220 After all edit files are replayed, the contents of the MemStore are written to disk (HFile) and the edit files are deleted.
1223 ===== Handling of Errors During Log Splitting
1225 If you set the `hbase.hlog.split.skip.errors` option to `true`, errors are treated as follows:
1227 * Any error encountered during splitting will be logged.
1228 * The problematic WAL log will be moved into the _.corrupt_ directory under the hbase `rootdir`,
1229 * Processing of the WAL will continue
1231 If the `hbase.hlog.split.skip.errors` option is set to `false`, the default, the exception will be propagated and the split will be logged as failed.
1232 See link:https://issues.apache.org/jira/browse/HBASE-2958[HBASE-2958 When
1233 hbase.hlog.split.skip.errors is set to false, we fail the split but that's it].
1234 We need to do more than just fail split if this flag is set.
1236 ====== How EOFExceptions are treated when splitting a crashed RegionServer's WALs
1238 If an EOFException occurs while splitting logs, the split proceeds even when `hbase.hlog.split.skip.errors` is set to `false`.
1239 An EOFException while reading the last log in the set of files to split is likely, because the RegionServer was likely in the process of writing a record at the time of a crash.
1240 For background, see link:https://issues.apache.org/jira/browse/HBASE-2643[HBASE-2643 Figure how to deal with eof splitting logs]
1242 ===== Performance Improvements during Log Splitting
1244 WAL log splitting and recovery can be resource intensive and take a long time, depending on the number of RegionServers involved in the crash and the size of the regions. <<distributed.log.splitting>> was developed to improve performance during log splitting.
1246 [[distributed.log.splitting]]
1247 .Enabling or Disabling Distributed Log Splitting
1249 Distributed log processing is enabled by default since HBase 0.92.
1250 The setting is controlled by the `hbase.master.distributed.log.splitting` property, which can be set to `true` or `false`, but defaults to `true`.
1252 ==== WAL splitting based on procedureV2
1253 After HBASE-20610, we introduce a new way to do WAL splitting coordination by procedureV2 framework. This can simplify the process of WAL splitting and no need to connect zookeeper any more.
1257 Currently, splitting WAL processes are coordinated by zookeeper. Each region server are trying to grab tasks from zookeeper. And the burden becomes heavier when the number of region server increase.
1259 [[implementation.on.master.side]]
1260 .Implementation on Master side
1261 During ServerCrashProcedure, SplitWALManager will create one SplitWALProcedure for each WAL file which should be split. Then each SplitWALProcedure will spawn a SplitWalRemoteProcedure to send the request to region server.
1262 SplitWALProcedure is a StateMachineProcedure and here is the state transfer diagram.
1264 .WAL_splitting_coordination
1265 image::WAL_splitting.png[]
1267 [[implementation.on.region.server.side]]
1268 .Implementation on Region Server side
1269 Region Server will receive a SplitWALCallable and execute it, which is much more straightforward than before. It will return null if success and return exception if there is any error.
1273 According to tests on a cluster which has 5 regionserver and 1 master.
1274 procedureV2 coordinated WAL splitting has a better performance than ZK coordinated WAL splitting no master when restarting the whole cluster or one region server crashing.
1276 [[enable.this.feature]]
1277 .Enable this feature
1278 To enable this feature, first we should ensure our package of HBase already contains these code. If not, please upgrade the package of HBase cluster without any configuration change first.
1279 Then change configuration 'hbase.split.wal.zk.coordinated' to false. Rolling upgrade the master with new configuration. Now WAL splitting are handled by our new implementation.
1280 But region server are still trying to grab tasks from zookeeper, we can rolling upgrade the region servers with the new configuration to stop that.
1283 ** Upgrade whole cluster to get the new Implementation.
1284 ** Upgrade Master with new configuration 'hbase.split.wal.zk.coordinated'=false.
1285 ** Upgrade region server to stop grab tasks from zookeeper.
1288 ==== WAL Compression ====
1290 The content of the WAL can be compressed using LRU Dictionary compression.
1291 This can be used to speed up WAL replication to different datanodes.
1292 The dictionary can store up to 2^15^ elements; eviction starts after this number is exceeded.
1294 To enable WAL compression, set the `hbase.regionserver.wal.enablecompression` property to `true`.
1295 The default value for this property is `false`.
1296 By default, WAL tag compression is turned on when WAL compression is enabled.
1297 You can turn off WAL tag compression by setting the `hbase.regionserver.wal.tags.enablecompression` property to 'false'.
1299 A possible downside to WAL compression is that we lose more data from the last block in the WAL if it ill-terminated
1300 mid-write. If entries in this last block were added with new dictionary entries but we failed persist the amended
1301 dictionary because of an abrupt termination, a read of this last block may not be able to resolve last-written entries.
1305 It is possible to set _durability_ on each Mutation or on a Table basis. Options include:
1307 * _SKIP_WAL_: Do not write Mutations to the WAL (See the next section, <<wal.disable>>).
1308 * _ASYNC_WAL_: Write the WAL asynchronously; do not hold-up clients waiting on the sync of their write to the filesystem but return immediately. The edit becomes visible. Meanwhile, in the background, the Mutation will be flushed to the WAL at some time later. This option currently may lose data. See HBASE-16689.
1309 * _SYNC_WAL_: The *default*. Each edit is sync'd to HDFS before we return success to the client.
1310 * _FSYNC_WAL_: Each edit is fsync'd to HDFS and the filesystem before we return success to the client.
1312 Do not confuse the _ASYNC_WAL_ option on a Mutation or Table with the _AsyncFSWAL_ writer; they are distinct
1313 options unfortunately closely named
1316 ==== Disabling the WAL
1318 It is possible to disable the WAL, to improve performance in certain specific situations.
1319 However, disabling the WAL puts your data at risk.
1320 The only situation where this is recommended is during a bulk load.
1321 This is because, in the event of a problem, the bulk load can be re-run with no risk of data loss.
1323 The WAL is disabled by calling the HBase client field `Mutation.writeToWAL(false)`.
1324 Use the `Mutation.setDurability(Durability.SKIP_WAL)` and Mutation.getDurability() methods to set and get the field's value.
1325 There is no way to disable the WAL for only a specific table.
1327 WARNING: If you disable the WAL for anything other than bulk loads, your data is at risk.
1333 Regions are the basic element of availability and distribution for tables, and are comprised of a Store per Column Family.
1334 The hierarchy of objects is as follows:
1338 Region (Regions for the table)
1339 Store (Store per ColumnFamily for each Region for the table)
1340 MemStore (MemStore for each Store for each Region for the table)
1341 StoreFile (StoreFiles for each Store for each Region for the table)
1342 Block (Blocks within a StoreFile within a Store for each Region for the table)
1345 For a description of what HBase files look like when written to HDFS, see <<trouble.namenode.hbase.objects>>.
1347 [[arch.regions.size]]
1348 === Considerations for Number of Regions
1350 In general, HBase is designed to run with a small (20-200) number of relatively large (5-20Gb) regions per server.
1351 The considerations for this are as follows:
1353 [[too_many_regions]]
1354 ==== Why should I keep my Region count low?
1356 Typically you want to keep your region count low on HBase for numerous reasons.
1357 Usually right around 100 regions per RegionServer has yielded the best results.
1358 Here are some of the reasons below for keeping region count low:
1360 . MSLAB (MemStore-local allocation buffer) requires 2MB per MemStore (that's 2MB per family per region). 1000 regions that have 2 families each is 3.9GB of heap used, and it's not even storing data yet.
1361 NB: the 2MB value is configurable.
1362 . If you fill all the regions at somewhat the same rate, the global memory usage makes it that it forces tiny flushes when you have too many regions which in turn generates compactions.
1363 Rewriting the same data tens of times is the last thing you want.
1364 An example is filling 1000 regions (with one family) equally and let's consider a lower bound for global MemStore usage of 5GB (the region server would have a big heap). Once it reaches 5GB it will force flush the biggest region, at that point they should almost all have about 5MB of data so it would flush that amount.
1365 5MB inserted later, it would flush another region that will now have a bit over 5MB of data, and so on.
1366 This is currently the main limiting factor for the number of regions; see <<ops.capacity.regions.count>> for detailed formula.
1367 . The master as is is allergic to tons of regions, and will take a lot of time assigning them and moving them around in batches.
1368 The reason is that it's heavy on ZK usage, and it's not very async at the moment (could really be improved -- and has been improved a bunch in 0.96 HBase).
1369 . In older versions of HBase (pre-HFile v2, 0.90 and previous), tons of regions on a few RS can cause the store file index to rise, increasing heap usage and potentially creating memory pressure or OOME on the RSs
1371 Another issue is the effect of the number of regions on MapReduce jobs; it is typical to have one mapper per HBase region.
1372 Thus, hosting only 5 regions per RS may not be enough to get sufficient number of tasks for a MapReduce job, while 1000 regions will generate far too many tasks.
1374 See <<ops.capacity.regions>> for configuration guidelines.
1376 [[regions.arch.assignment]]
1377 === Region-RegionServer Assignment
1379 This section describes how Regions are assigned to RegionServers.
1381 [[regions.arch.assignment.startup]]
1384 When HBase starts regions are assigned as follows (short version):
1386 . The Master invokes the `AssignmentManager` upon startup.
1387 . The `AssignmentManager` looks at the existing region assignments in `hbase:meta`.
1388 . If the region assignment is still valid (i.e., if the RegionServer is still online) then the assignment is kept.
1389 . If the assignment is invalid, then the `LoadBalancerFactory` is invoked to assign the region.
1390 The load balancer (`StochasticLoadBalancer` by default in HBase 1.0) assign the region to a RegionServer.
1391 . `hbase:meta` is updated with the RegionServer assignment (if needed) and the RegionServer start codes (start time of the RegionServer process) upon region opening by the RegionServer.
1393 [[regions.arch.assignment.failover]]
1396 When a RegionServer fails:
1398 . The regions immediately become unavailable because the RegionServer is down.
1399 . The Master will detect that the RegionServer has failed.
1400 . The region assignments will be considered invalid and will be re-assigned just like the startup sequence.
1401 . In-flight queries are re-tried, and not lost.
1402 . Operations are switched to a new RegionServer within the following amount of time:
1406 ZooKeeper session timeout + split time + assignment/replay time
1410 [[regions.arch.balancer]]
1411 ==== Region Load Balancing
1413 Regions can be periodically moved by the <<master.processes.loadbalancer>>.
1415 [[regions.arch.states]]
1416 ==== Region State Transition
1418 HBase maintains a state for each region and persists the state in `hbase:meta`.
1419 The state of the `hbase:meta` region itself is persisted in ZooKeeper.
1420 You can see the states of regions in transition in the Master web UI.
1421 Following is the list of possible region states.
1423 .Possible Region States
1424 * `OFFLINE`: the region is offline and not opening
1425 * `OPENING`: the region is in the process of being opened
1426 * `OPEN`: the region is open and the RegionServer has notified the master
1427 * `FAILED_OPEN`: the RegionServer failed to open the region
1428 * `CLOSING`: the region is in the process of being closed
1429 * `CLOSED`: the RegionServer has closed the region and notified the master
1430 * `FAILED_CLOSE`: the RegionServer failed to close the region
1431 * `SPLITTING`: the RegionServer notified the master that the region is splitting
1432 * `SPLIT`: the RegionServer notified the master that the region has finished splitting
1433 * `SPLITTING_NEW`: this region is being created by a split which is in progress
1434 * `MERGING`: the RegionServer notified the master that this region is being merged with another region
1435 * `MERGED`: the RegionServer notified the master that this region has been merged
1436 * `MERGING_NEW`: this region is being created by a merge of two regions
1438 .Region State Transitions
1439 image::region_states.png[]
1442 * Brown: Offline state, a special state that can be transient (after closed before opening), terminal (regions of disabled tables), or initial (regions of newly created tables)
1443 * Palegreen: Online state that regions can serve requests
1444 * Lightblue: Transient states
1445 * Red: Failure states that need OPS attention
1446 * Gold: Terminal states of regions split/merged
1447 * Grey: Initial states of regions created through split/merge
1449 .Transition State Descriptions
1450 . The master moves a region from `OFFLINE` to `OPENING` state and tries to assign the region to a RegionServer.
1451 The RegionServer may or may not have received the open region request.
1452 The master retries sending the open region request to the RegionServer until the RPC goes through or the master runs out of retries.
1453 After the RegionServer receives the open region request, the RegionServer begins opening the region.
1454 . If the master is running out of retries, the master prevents the RegionServer from opening the region by moving the region to `CLOSING` state and trying to close it, even if the RegionServer is starting to open the region.
1455 . After the RegionServer opens the region, it continues to try to notify the master until the master moves the region to `OPEN` state and notifies the RegionServer.
1456 The region is now open.
1457 . If the RegionServer cannot open the region, it notifies the master.
1458 The master moves the region to `CLOSED` state and tries to open the region on a different RegionServer.
1459 . If the master cannot open the region on any of a certain number of regions, it moves the region to `FAILED_OPEN` state, and takes no further action until an operator intervenes from the HBase shell, or the server is dead.
1460 . The master moves a region from `OPEN` to `CLOSING` state.
1461 The RegionServer holding the region may or may not have received the close region request.
1462 The master retries sending the close request to the server until the RPC goes through or the master runs out of retries.
1463 . If the RegionServer is not online, or throws `NotServingRegionException`, the master moves the region to `OFFLINE` state and re-assigns it to a different RegionServer.
1464 . If the RegionServer is online, but not reachable after the master runs out of retries, the master moves the region to `FAILED_CLOSE` state and takes no further action until an operator intervenes from the HBase shell, or the server is dead.
1465 . If the RegionServer gets the close region request, it closes the region and notifies the master.
1466 The master moves the region to `CLOSED` state and re-assigns it to a different RegionServer.
1467 . Before assigning a region, the master moves the region to `OFFLINE` state automatically if it is in `CLOSED` state.
1468 . When a RegionServer is about to split a region, it notifies the master.
1469 The master moves the region to be split from `OPEN` to `SPLITTING` state and add the two new regions to be created to the RegionServer.
1470 These two regions are in `SPLITTING_NEW` state initially.
1471 . After notifying the master, the RegionServer starts to split the region.
1472 Once past the point of no return, the RegionServer notifies the master again so the master can update the `hbase:meta` table.
1473 However, the master does not update the region states until it is notified by the server that the split is done.
1474 If the split is successful, the splitting region is moved from `SPLITTING` to `SPLIT` state and the two new regions are moved from `SPLITTING_NEW` to `OPEN` state.
1475 . If the split fails, the splitting region is moved from `SPLITTING` back to `OPEN` state, and the two new regions which were created are moved from `SPLITTING_NEW` to `OFFLINE` state.
1476 . When a RegionServer is about to merge two regions, it notifies the master first.
1477 The master moves the two regions to be merged from `OPEN` to `MERGING` state, and adds the new region which will hold the contents of the merged regions region to the RegionServer.
1478 The new region is in `MERGING_NEW` state initially.
1479 . After notifying the master, the RegionServer starts to merge the two regions.
1480 Once past the point of no return, the RegionServer notifies the master again so the master can update the META.
1481 However, the master does not update the region states until it is notified by the RegionServer that the merge has completed.
1482 If the merge is successful, the two merging regions are moved from `MERGING` to `MERGED` state and the new region is moved from `MERGING_NEW` to `OPEN` state.
1483 . If the merge fails, the two merging regions are moved from `MERGING` back to `OPEN` state, and the new region which was created to hold the contents of the merged regions is moved from `MERGING_NEW` to `OFFLINE` state.
1484 . For regions in `FAILED_OPEN` or `FAILED_CLOSE` states, the master tries to close them again when they are reassigned by an operator via HBase Shell.
1486 [[regions.arch.locality]]
1487 === Region-RegionServer Locality
1489 Over time, Region-RegionServer locality is achieved via HDFS block replication.
1490 The HDFS client does the following by default when choosing locations to write replicas:
1492 . First replica is written to local node
1493 . Second replica is written to a random node on another rack
1494 . Third replica is written on the same rack as the second, but on a different node chosen randomly
1495 . Subsequent replicas are written on random nodes on the cluster.
1496 See _Replica Placement: The First Baby Steps_ on this page: link:https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html[HDFS Architecture]
1498 Thus, HBase eventually achieves locality for a region after a flush or a compaction.
1499 In a RegionServer failover situation a RegionServer may be assigned regions with non-local StoreFiles (because none of the replicas are local), however as new data is written in the region, or the table is compacted and StoreFiles are re-written, they will become "local" to the RegionServer.
1501 For more information, see _Replica Placement: The First Baby Steps_ on this page: link:https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html[HDFS Architecture] and also Lars George's blog on link:http://www.larsgeorge.com/2010/05/hbase-file-locality-in-hdfs.html[HBase and HDFS locality].
1503 [[arch.region.splits]]
1506 Regions split when they reach a configured threshold.
1507 Below we treat the topic in short.
1508 For a longer exposition, see link:http://hortonworks.com/blog/apache-hbase-region-splitting-and-merging/[Apache HBase Region Splitting and Merging] by our Enis Soztutar.
1510 Splits run unaided on the RegionServer; i.e. the Master does not participate.
1511 The RegionServer splits a region, offlines the split region and then adds the daughter regions to `hbase:meta`, opens daughters on the parent's hosting RegionServer and then reports the split to the Master.
1512 See <<disable.splitting>> for how to manually manage splits (and for why you might do this).
1514 ==== Custom Split Policies
1515 You can override the default split policy using a custom
1516 link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.html[RegionSplitPolicy](HBase 0.94+).
1517 Typically a custom split policy should extend HBase's default split policy:
1518 link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.html[IncreasingToUpperBoundRegionSplitPolicy].
1520 The policy can set globally through the HBase configuration or on a per-table
1523 .Configuring the Split Policy Globally in _hbase-site.xml_
1527 <name>hbase.regionserver.region.split.policy</name>
1528 <value>org.apache.hadoop.hbase.regionserver.IncreasingToUpperBoundRegionSplitPolicy</value>
1532 .Configuring a Split Policy On a Table Using the Java API
1534 HTableDescriptor tableDesc = new HTableDescriptor("test");
1535 tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, ConstantSizeRegionSplitPolicy.class.getName());
1536 tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes("cf1")));
1537 admin.createTable(tableDesc);
1541 .Configuring the Split Policy On a Table Using HBase Shell
1543 hbase> create 'test', {METADATA => {'SPLIT_POLICY' => 'org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy'}},{NAME => 'cf1'}
1546 The policy can be set globally through the HBaseConfiguration used or on a per table basis:
1549 HTableDescriptor myHtd = ...;
1550 myHtd.setValue(HTableDescriptor.SPLIT_POLICY, MyCustomSplitPolicy.class.getName());
1553 NOTE: The `DisabledRegionSplitPolicy` policy blocks manual region splitting.
1555 [[manual_region_splitting_decisions]]
1556 === Manual Region Splitting
1558 It is possible to manually split your table, either at table creation (pre-splitting), or at a later time as an administrative action.
1559 You might choose to split your region for one or more of the following reasons.
1560 There may be other valid reasons, but the need to manually split your table might also point to problems with your schema design.
1562 .Reasons to Manually Split Your Table
1563 * Your data is sorted by timeseries or another similar algorithm that sorts new data at the end of the table.
1564 This means that the Region Server holding the last region is always under load, and the other Region Servers are idle, or mostly idle.
1565 See also <<timeseries>>.
1566 * You have developed an unexpected hotspot in one region of your table.
1567 For instance, an application which tracks web searches might be inundated by a lot of searches for a celebrity in the event of news about that celebrity.
1568 See <<perf.one.region,perf.one.region>> for more discussion about this particular scenario.
1569 * After a big increase in the number of RegionServers in your cluster, to get the load spread out quickly.
1570 * Before a bulk-load which is likely to cause unusual and uneven load across regions.
1572 See <<disable.splitting>> for a discussion about the dangers and possible benefits of managing splitting completely manually.
1574 NOTE: The `DisabledRegionSplitPolicy` policy blocks manual region splitting.
1576 ==== Determining Split Points
1578 The goal of splitting your table manually is to improve the chances of balancing the load across the cluster in situations where good rowkey design alone won't get you there.
1579 Keeping that in mind, the way you split your regions is very dependent upon the characteristics of your data.
1580 It may be that you already know the best way to split your table.
1581 If not, the way you split your table depends on what your keys are like.
1583 Alphanumeric Rowkeys::
1584 If your rowkeys start with a letter or number, you can split your table at letter or number boundaries.
1585 For instance, the following command creates a table with regions that split at each vowel, so the first region has A-D, the second region has E-H, the third region has I-N, the fourth region has O-V, and the fifth region has U-Z.
1587 Using a Custom Algorithm::
1588 The RegionSplitter tool is provided with HBase, and uses a _SplitAlgorithm_ to determine split points for you.
1589 As parameters, you give it the algorithm, desired number of regions, and column families.
1590 It includes three split algorithms.
1592 `link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.HexStringSplit.html[HexStringSplit]`
1593 algorithm, which assumes the row keys are hexadecimal strings.
1595 `link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.DecimalStringSplit.html[DecimalStringSplit]`
1596 algorithm, which assumes the row keys are decimal strings in the range 00000000 to 99999999.
1598 `link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.UniformSplit.html[UniformSplit]`,
1599 assumes the row keys are random byte arrays.
1600 You will probably need to develop your own
1601 `link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.SplitAlgorithm.html[SplitAlgorithm]`,
1602 using the provided ones as models.
1604 === Online Region Merges
1606 Both Master and RegionServer participate in the event of online region merges.
1607 Client sends merge RPC to the master, then the master moves the regions together to the RegionServer where the more heavily loaded region resided. Finally the master sends the merge request to this RegionServer which then runs the merge.
1608 Similar to process of region splitting, region merges run as a local transaction on the RegionServer. It offlines the regions and then merges two regions on the file system, atomically delete merging regions from `hbase:meta` and adds the merged region to `hbase:meta`, opens the merged region on the RegionServer and reports the merge to the Master.
1610 An example of region merges in the HBase shell
1613 $ hbase> merge_region 'ENCODED_REGIONNAME', 'ENCODED_REGIONNAME'
1614 $ hbase> merge_region 'ENCODED_REGIONNAME', 'ENCODED_REGIONNAME', true
1616 It's an asynchronous operation and call returns immediately without waiting merge completed.
1617 Passing `true` as the optional third parameter will force a merge. Normally only adjacent regions can be merged.
1618 The `force` parameter overrides this behaviour and is for expert use only.
1623 A Store hosts a MemStore and 0 or more StoreFiles (HFiles). A Store corresponds to a column family for a table for a given region.
1628 The MemStore holds in-memory modifications to the Store.
1629 Modifications are Cells/KeyValues.
1630 When a flush is requested, the current MemStore is moved to a snapshot and is cleared.
1631 HBase continues to serve edits from the new MemStore and backing snapshot until the flusher reports that the flush succeeded.
1632 At this point, the snapshot is discarded.
1633 Note that when the flush happens, MemStores that belong to the same region will all be flushed.
1637 A MemStore flush can be triggered under any of the conditions listed below.
1638 The minimum flush unit is per region, not at individual MemStore level.
1640 . When a MemStore reaches the size specified by `hbase.hregion.memstore.flush.size`,
1641 all MemStores that belong to its region will be flushed out to disk.
1643 . When the overall MemStore usage reaches the value specified by
1644 `hbase.regionserver.global.memstore.upperLimit`, MemStores from various regions
1645 will be flushed out to disk to reduce overall MemStore usage in a RegionServer.
1647 The flush order is based on the descending order of a region's MemStore usage.
1649 Regions will have their MemStores flushed until the overall MemStore usage drops
1650 to or slightly below `hbase.regionserver.global.memstore.lowerLimit`.
1652 . When the number of WAL log entries in a given region server's WAL reaches the
1653 value specified in `hbase.regionserver.max.logs`, MemStores from various regions
1654 will be flushed out to disk to reduce the number of logs in the WAL.
1656 The flush order is based on time.
1658 Regions with the oldest MemStores are flushed first until WAL count drops below
1659 `hbase.regionserver.max.logs`.
1664 * When a client issues a scan against a table, HBase generates `RegionScanner` objects, one per region, to serve the scan request.
1665 * The `RegionScanner` object contains a list of `StoreScanner` objects, one per column family.
1666 * Each `StoreScanner` object further contains a list of `StoreFileScanner` objects, corresponding to each StoreFile and HFile of the corresponding column family, and a list of `KeyValueScanner` objects for the MemStore.
1667 * The two lists are merged into one, which is sorted in ascending order with the scan object for the MemStore at the end of the list.
1668 * When a `StoreFileScanner` object is constructed, it is associated with a `MultiVersionConcurrencyControl` read point, which is the current `memstoreTS`, filtering out any new updates beyond the read point.
1671 ==== StoreFile (HFile)
1673 StoreFiles are where your data lives.
1677 The _HFile_ file format is based on the SSTable file described in the link:http://research.google.com/archive/bigtable.html[BigTable [2006]] paper and on Hadoop's link:https://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/file/tfile/TFile.html[TFile] (The unit test suite and the compression harness were taken directly from TFile). Schubert Zhang's blog post on link:http://cloudepr.blogspot.com/2009/09/hfile-block-indexed-file-format-to.html[HFile: A Block-Indexed File Format to Store Sorted Key-Value Pairs] makes for a thorough introduction to HBase's HFile.
1678 Matteo Bertozzi has also put up a helpful description, link:http://th30z.blogspot.com/2011/02/hbase-io-hfile.html?spref=tw[HBase I/O: HFile].
1680 For more information, see the HFile source code.
1681 Also see <<hfilev2>> for information about the HFile v2 format that was included in 0.92.
1686 To view a textualized version of HFile content, you can use the `hbase hfile` tool.
1687 Type the following to see usage:
1691 $ ${HBASE_HOME}/bin/hbase hfile
1693 For example, to view the content of the file _hdfs://10.81.47.41:8020/hbase/default/TEST/1418428042/DSMP/4759508618286845475_, type the following:
1696 $ ${HBASE_HOME}/bin/hbase hfile -v -f hdfs://10.81.47.41:8020/hbase/default/TEST/1418428042/DSMP/4759508618286845475
1698 If you leave off the option -v to see just a summary on the HFile.
1699 See usage for other things to do with the `hfile` tool.
1701 NOTE: In the output of this tool, you might see 'seqid=0' for certain keys in places such as 'Mid-key'/'firstKey'/'lastKey'. These are
1702 'KeyOnlyKeyValue' type instances - meaning their seqid is irrelevant & we just need the keys of these Key-Value instances.
1705 ===== StoreFile Directory Structure on HDFS
1707 For more information of what StoreFiles look like on HDFS with respect to the directory structure, see <<trouble.namenode.hbase.objects>>.
1712 StoreFiles are composed of blocks.
1713 The blocksize is configured on a per-ColumnFamily basis.
1715 Compression happens at the block level within StoreFiles.
1716 For more information on compression, see <<compression>>.
1718 For more information on blocks, see the HFileBlock source code.
1723 The KeyValue class is the heart of data storage in HBase.
1724 KeyValue wraps a byte array and takes offsets and lengths into the passed array which specify where to start interpreting the content as KeyValue.
1726 The KeyValue format inside a byte array is:
1733 The Key is further decomposed as:
1736 * row (i.e., the rowkey)
1737 * columnfamilylength
1741 * keytype (e.g., Put, Delete, DeleteColumn, DeleteFamily)
1743 KeyValue instances are _not_ split across blocks.
1744 For example, if there is an 8 MB KeyValue, even if the block-size is 64kb this KeyValue will be read in as a coherent block.
1745 For more information, see the KeyValue source code.
1747 [[keyvalue.example]]
1750 To emphasize the points above, examine what happens with two Puts for two different columns for the same row:
1752 * Put #1: `rowkey=row1, cf:attr1=value1`
1753 * Put #2: `rowkey=row1, cf:attr2=value2`
1755 Even though these are for the same row, a KeyValue is created for each column:
1757 Key portion for Put #1:
1759 * `rowlength ------------> 4`
1760 * `row ------------------> row1`
1761 * `columnfamilylength ---> 2`
1762 * `columnfamily ---------> cf`
1763 * `columnqualifier ------> attr1`
1764 * `timestamp ------------> server time of Put`
1765 * `keytype --------------> Put`
1767 Key portion for Put #2:
1769 * `rowlength ------------> 4`
1770 * `row ------------------> row1`
1771 * `columnfamilylength ---> 2`
1772 * `columnfamily ---------> cf`
1773 * `columnqualifier ------> attr2`
1774 * `timestamp ------------> server time of Put`
1775 * `keytype --------------> Put`
1777 It is critical to understand that the rowkey, ColumnFamily, and column (aka columnqualifier) are embedded within the KeyValue instance.
1778 The longer these identifiers are, the bigger the KeyValue is.
1783 .Ambiguous Terminology
1784 * A _StoreFile_ is a facade of HFile.
1785 In terms of compaction, use of StoreFile seems to have prevailed in the past.
1786 * A _Store_ is the same thing as a ColumnFamily.
1787 StoreFiles are related to a Store, or ColumnFamily.
1788 * If you want to read more about StoreFiles versus HFiles and Stores versus ColumnFamilies, see link:https://issues.apache.org/jira/browse/HBASE-11316[HBASE-11316].
1790 When the MemStore reaches a given size (`hbase.hregion.memstore.flush.size`), it flushes its contents to a StoreFile.
1791 The number of StoreFiles in a Store increases over time. _Compaction_ is an operation which reduces the number of StoreFiles in a Store, by merging them together, in order to increase performance on read operations.
1792 Compactions can be resource-intensive to perform, and can either help or hinder performance depending on many factors.
1794 Compactions fall into two categories: minor and major.
1795 Minor and major compactions differ in the following ways.
1797 _Minor compactions_ usually select a small number of small, adjacent StoreFiles and rewrite them as a single StoreFile.
1798 Minor compactions do not drop (filter out) deletes or expired versions, because of potential side effects.
1799 See <<compaction.and.deletes>> and <<compaction.and.versions>> for information on how deletes and versions are handled in relation to compactions.
1800 The end result of a minor compaction is fewer, larger StoreFiles for a given Store.
1802 The end result of a _major compaction_ is a single StoreFile per Store.
1803 Major compactions also process delete markers and max versions.
1804 See <<compaction.and.deletes>> and <<compaction.and.versions>> for information on how deletes and versions are handled in relation to compactions.
1806 [[compaction.and.deletes]]
1807 .Compaction and Deletions
1808 When an explicit deletion occurs in HBase, the data is not actually deleted.
1809 Instead, a _tombstone_ marker is written.
1810 The tombstone marker prevents the data from being returned with queries.
1811 During a major compaction, the data is actually deleted, and the tombstone marker is removed from the StoreFile.
1812 If the deletion happens because of an expired TTL, no tombstone is created.
1813 Instead, the expired data is filtered out and is not written back to the compacted StoreFile.
1815 [[compaction.and.versions]]
1816 .Compaction and Versions
1817 When you create a Column Family, you can specify the maximum number of versions to keep, by specifying `HColumnDescriptor.setMaxVersions(int versions)`.
1818 The default value is `3`.
1819 If more versions than the specified maximum exist, the excess versions are filtered out and not written back to the compacted StoreFile.
1821 .Major Compactions Can Impact Query Results
1824 In some situations, older versions can be inadvertently resurrected if a newer version is explicitly deleted.
1825 See <<major.compactions.change.query.results>> for a more in-depth explanation.
1826 This situation is only possible before the compaction finishes.
1829 In theory, major compactions improve performance.
1830 However, on a highly loaded system, major compactions can require an inappropriate number of resources and adversely affect performance.
1831 In a default configuration, major compactions are scheduled automatically to run once in a 7-day period.
1832 This is sometimes inappropriate for systems in production.
1833 You can manage major compactions manually.
1834 See <<managed.compactions>>.
1836 Compactions do not perform region merges.
1837 See <<ops.regionmgt.merge>> for more information on region merging.
1840 We can switch on and off the compactions at region servers. Switching off compactions will also
1841 interrupt any currently ongoing compactions. It can be done dynamically using the "compaction_switch"
1842 command from hbase shell. If done from the command line, this setting will be lost on restart of the
1843 server. To persist the changes across region servers modify the configuration hbase.regionserver
1844 .compaction.enabled in hbase-site.xml and restart HBase.
1847 [[compaction.file.selection]]
1848 ===== Compaction Policy - HBase 0.96.x and newer
1850 Compacting large StoreFiles, or too many StoreFiles at once, can cause more IO load than your cluster is able to handle without causing performance problems.
1851 The method by which HBase selects which StoreFiles to include in a compaction (and whether the compaction is a minor or major compaction) is called the _compaction policy_.
1853 Prior to HBase 0.96.x, there was only one compaction policy.
1854 That original compaction policy is still available as `RatioBasedCompactionPolicy`. The new compaction default policy, called `ExploringCompactionPolicy`, was subsequently backported to HBase 0.94 and HBase 0.95, and is the default in HBase 0.96 and newer.
1855 It was implemented in link:https://issues.apache.org/jira/browse/HBASE-7842[HBASE-7842].
1856 In short, `ExploringCompactionPolicy` attempts to select the best possible set of StoreFiles to compact with the least amount of work, while the `RatioBasedCompactionPolicy` selects the first set that meets the criteria.
1858 Regardless of the compaction policy used, file selection is controlled by several configurable parameters and happens in a multi-step approach.
1859 These parameters will be explained in context, and then will be given in a table which shows their descriptions, defaults, and implications of changing them.
1861 [[compaction.being.stuck]]
1864 When the MemStore gets too large, it needs to flush its contents to a StoreFile.
1865 However, Stores are configured with a bound on the number StoreFiles,
1866 `hbase.hstore.blockingStoreFiles`, and if in excess, the MemStore flush must wait
1867 until the StoreFile count is reduced by one or more compactions. If the MemStore
1868 is too large and the number of StoreFiles is also too high, the algorithm is said
1869 to be "stuck". By default we'll wait on compactions up to
1870 `hbase.hstore.blockingWaitTime` milliseconds. If this period expires, we'll flush
1871 anyways even though we are in excess of the
1872 `hbase.hstore.blockingStoreFiles` count.
1874 Upping the `hbase.hstore.blockingStoreFiles` count will allow flushes to happen
1875 but a Store with many StoreFiles in will likely have higher read latencies. Try to
1876 figure why Compactions are not keeping up. Is it a write spurt that is bringing
1877 about this situation or is a regular occurance and the cluster is under-provisioned
1878 for the volume of writes?
1880 [[exploringcompaction.policy]]
1881 ====== The ExploringCompactionPolicy Algorithm
1883 The ExploringCompactionPolicy algorithm considers each possible set of adjacent StoreFiles before choosing the set where compaction will have the most benefit.
1885 One situation where the ExploringCompactionPolicy works especially well is when you are bulk-loading data and the bulk loads create larger StoreFiles than the StoreFiles which are holding data older than the bulk-loaded data.
1886 This can "trick" HBase into choosing to perform a major compaction each time a compaction is needed, and cause a lot of extra overhead.
1887 With the ExploringCompactionPolicy, major compactions happen much less frequently because minor compactions are more efficient.
1889 In general, ExploringCompactionPolicy is the right choice for most situations, and thus is the default compaction policy.
1890 You can also use ExploringCompactionPolicy along with <<ops.stripe>>.
1892 The logic of this policy can be examined in hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java.
1893 The following is a walk-through of the logic of the ExploringCompactionPolicy.
1896 . Make a list of all existing StoreFiles in the Store.
1897 The rest of the algorithm filters this list to come up with the subset of HFiles which will be chosen for compaction.
1898 . If this was a user-requested compaction, attempt to perform the requested compaction type, regardless of what would normally be chosen.
1899 Note that even if the user requests a major compaction, it may not be possible to perform a major compaction.
1900 This may be because not all StoreFiles in the Column Family are available to compact or because there are too many Stores in the Column Family.
1901 . Some StoreFiles are automatically excluded from consideration.
1904 * StoreFiles that are larger than `hbase.hstore.compaction.max.size`
1905 * StoreFiles that were created by a bulk-load operation which explicitly excluded compaction.
1906 You may decide to exclude StoreFiles resulting from bulk loads, from compaction.
1907 To do this, specify the `hbase.mapreduce.hfileoutputformat.compaction.exclude` parameter during the bulk load operation.
1909 . Iterate through the list from step 1, and make a list of all potential sets of StoreFiles to compact together.
1910 A potential set is a grouping of `hbase.hstore.compaction.min` contiguous StoreFiles in the list.
1911 For each set, perform some sanity-checking and figure out whether this is the best compaction that could be done:
1913 * If the number of StoreFiles in this set (not the size of the StoreFiles) is fewer than `hbase.hstore.compaction.min` or more than `hbase.hstore.compaction.max`, take it out of consideration.
1914 * Compare the size of this set of StoreFiles with the size of the smallest possible compaction that has been found in the list so far.
1915 If the size of this set of StoreFiles represents the smallest compaction that could be done, store it to be used as a fall-back if the algorithm is "stuck" and no StoreFiles would otherwise be chosen.
1916 See <<compaction.being.stuck>>.
1917 * Do size-based sanity checks against each StoreFile in this set of StoreFiles.
1918 ** If the size of this StoreFile is larger than `hbase.hstore.compaction.max.size`, take it out of consideration.
1919 ** If the size is greater than or equal to `hbase.hstore.compaction.min.size`, sanity-check it against the file-based ratio to see whether it is too large to be considered.
1921 The sanity-checking is successful if:
1922 ** There is only one StoreFile in this set, or
1923 ** For each StoreFile, its size multiplied by `hbase.hstore.compaction.ratio` (or `hbase.hstore.compaction.ratio.offpeak` if off-peak hours are configured and it is during off-peak hours) is less than the sum of the sizes of the other HFiles in the set.
1925 . If this set of StoreFiles is still in consideration, compare it to the previously-selected best compaction.
1926 If it is better, replace the previously-selected best compaction with this one.
1927 . When the entire list of potential compactions has been processed, perform the best compaction that was found.
1928 If no StoreFiles were selected for compaction, but there are multiple StoreFiles, assume the algorithm is stuck (see <<compaction.being.stuck>>) and if so, perform the smallest compaction that was found in step 3.
1930 [[compaction.ratiobasedcompactionpolicy.algorithm]]
1931 ====== RatioBasedCompactionPolicy Algorithm
1933 The RatioBasedCompactionPolicy was the only compaction policy prior to HBase 0.96, though ExploringCompactionPolicy has now been backported to HBase 0.94 and 0.95.
1934 To use the RatioBasedCompactionPolicy rather than the ExploringCompactionPolicy, set `hbase.hstore.defaultengine.compactionpolicy.class` to `RatioBasedCompactionPolicy` in the _hbase-site.xml_ file.
1935 To switch back to the ExploringCompactionPolicy, remove the setting from the _hbase-site.xml_.
1937 The following section walks you through the algorithm used to select StoreFiles for compaction in the RatioBasedCompactionPolicy.
1940 . The first phase is to create a list of all candidates for compaction.
1941 A list is created of all StoreFiles not already in the compaction queue, and all StoreFiles newer than the newest file that is currently being compacted.
1942 This list of StoreFiles is ordered by the sequence ID.
1943 The sequence ID is generated when a Put is appended to the write-ahead log (WAL), and is stored in the metadata of the HFile.
1944 . Check to see if the algorithm is stuck (see <<compaction.being.stuck>>, and if so, a major compaction is forced.
1945 This is a key area where <<exploringcompaction.policy>> is often a better choice than the RatioBasedCompactionPolicy.
1946 . If the compaction was user-requested, try to perform the type of compaction that was requested.
1947 Note that a major compaction may not be possible if all HFiles are not available for compaction or if too many StoreFiles exist (more than `hbase.hstore.compaction.max`).
1948 . Some StoreFiles are automatically excluded from consideration.
1951 * StoreFiles that are larger than `hbase.hstore.compaction.max.size`
1952 * StoreFiles that were created by a bulk-load operation which explicitly excluded compaction.
1953 You may decide to exclude StoreFiles resulting from bulk loads, from compaction.
1954 To do this, specify the `hbase.mapreduce.hfileoutputformat.compaction.exclude` parameter during the bulk load operation.
1956 . The maximum number of StoreFiles allowed in a major compaction is controlled by the `hbase.hstore.compaction.max` parameter.
1957 If the list contains more than this number of StoreFiles, a minor compaction is performed even if a major compaction would otherwise have been done.
1958 However, a user-requested major compaction still occurs even if there are more than `hbase.hstore.compaction.max` StoreFiles to compact.
1959 . If the list contains fewer than `hbase.hstore.compaction.min` StoreFiles to compact, a minor compaction is aborted.
1960 Note that a major compaction can be performed on a single HFile.
1961 Its function is to remove deletes and expired versions, and reset locality on the StoreFile.
1962 . The value of the `hbase.hstore.compaction.ratio` parameter is multiplied by the sum of StoreFiles smaller than a given file, to determine whether that StoreFile is selected for compaction during a minor compaction.
1963 For instance, if hbase.hstore.compaction.ratio is 1.2, FileX is 5MB, FileY is 2MB, and FileZ is 3MB:
1966 5 <= 1.2 x (2 + 3) or 5 <= 6
1969 In this scenario, FileX is eligible for minor compaction.
1970 If FileX were 7MB, it would not be eligible for minor compaction.
1971 This ratio favors smaller StoreFile.
1972 You can configure a different ratio for use in off-peak hours, using the parameter `hbase.hstore.compaction.ratio.offpeak`, if you also configure `hbase.offpeak.start.hour` and `hbase.offpeak.end.hour`.
1974 . If the last major compaction was too long ago and there is more than one StoreFile to be compacted, a major compaction is run, even if it would otherwise have been minor.
1975 By default, the maximum time between major compactions is 7 days, plus or minus a 4.8 hour period, and determined randomly within those parameters.
1976 Prior to HBase 0.96, the major compaction period was 24 hours.
1977 See `hbase.hregion.majorcompaction` in the table below to tune or disable time-based major compactions.
1979 [[compaction.parameters]]
1980 ====== Parameters Used by Compaction Algorithm
1982 This table contains the main configuration parameters for compaction.
1983 This list is not exhaustive.
1984 To tune these parameters from the defaults, edit the _hbase-default.xml_ file.
1985 For a full list of all configuration parameters available, see <<config.files,config.files>>
1987 `hbase.hstore.compaction.min`::
1988 The minimum number of StoreFiles which must be eligible for compaction before compaction can run.
1989 The goal of tuning `hbase.hstore.compaction.min` is to avoid ending up with too many tiny StoreFiles
1990 to compact. Setting this value to 2 would cause a minor compaction each time you have two StoreFiles
1991 in a Store, and this is probably not appropriate. If you set this value too high, all the other
1992 values will need to be adjusted accordingly. For most cases, the default value is appropriate.
1993 In previous versions of HBase, the parameter `hbase.hstore.compaction.min` was called
1994 `hbase.hstore.compactionThreshold`.
1998 `hbase.hstore.compaction.max`::
1999 The maximum number of StoreFiles which will be selected for a single minor compaction,
2000 regardless of the number of eligible StoreFiles. Effectively, the value of
2001 `hbase.hstore.compaction.max` controls the length of time it takes a single
2002 compaction to complete. Setting it larger means that more StoreFiles are included
2003 in a compaction. For most cases, the default value is appropriate.
2007 `hbase.hstore.compaction.min.size`::
2008 A StoreFile smaller than this size will always be eligible for minor compaction.
2009 StoreFiles this size or larger are evaluated by `hbase.hstore.compaction.ratio`
2010 to determine if they are eligible. Because this limit represents the "automatic
2011 include" limit for all StoreFiles smaller than this value, this value may need
2012 to be reduced in write-heavy environments where many files in the 1-2 MB range
2013 are being flushed, because every StoreFile will be targeted for compaction and
2014 the resulting StoreFiles may still be under the minimum size and require further
2015 compaction. If this parameter is lowered, the ratio check is triggered more quickly.
2016 This addressed some issues seen in earlier versions of HBase but changing this
2017 parameter is no longer necessary in most situations.
2021 `hbase.hstore.compaction.max.size`::
2022 A StoreFile larger than this size will be excluded from compaction. The effect of
2023 raising `hbase.hstore.compaction.max.size` is fewer, larger StoreFiles that do not
2024 get compacted often. If you feel that compaction is happening too often without
2025 much benefit, you can try raising this value.
2027 *Default*: `Long.MAX_VALUE`
2029 `hbase.hstore.compaction.ratio`::
2030 For minor compaction, this ratio is used to determine whether a given StoreFile
2031 which is larger than `hbase.hstore.compaction.min.size` is eligible for compaction.
2032 Its effect is to limit compaction of large StoreFile. The value of
2033 `hbase.hstore.compaction.ratio` is expressed as a floating-point decimal.
2035 * A large ratio, such as 10, will produce a single giant StoreFile. Conversely,
2036 a value of .25, will produce behavior similar to the BigTable compaction algorithm,
2037 producing four StoreFiles.
2038 * A moderate value of between 1.0 and 1.4 is recommended. When tuning this value,
2039 you are balancing write costs with read costs. Raising the value (to something like
2040 1.4) will have more write costs, because you will compact larger StoreFiles.
2041 However, during reads, HBase will need to seek through fewer StoreFiles to
2042 accomplish the read. Consider this approach if you cannot take advantage of <<blooms>>.
2043 * Alternatively, you can lower this value to something like 1.0 to reduce the
2044 background cost of writes, and use to limit the number of StoreFiles touched
2045 during reads. For most cases, the default value is appropriate.
2049 `hbase.hstore.compaction.ratio.offpeak`::
2050 The compaction ratio used during off-peak compactions, if off-peak hours are
2051 also configured (see below). Expressed as a floating-point decimal. This allows
2052 for more aggressive (or less aggressive, if you set it lower than
2053 `hbase.hstore.compaction.ratio`) compaction during a set time period. Ignored
2054 if off-peak is disabled (default). This works the same as
2055 `hbase.hstore.compaction.ratio`.
2059 `hbase.offpeak.start.hour`::
2060 The start of off-peak hours, expressed as an integer between 0 and 23, inclusive.
2061 Set to -1 to disable off-peak.
2063 *Default*: `-1` (disabled)
2065 `hbase.offpeak.end.hour`::
2066 The end of off-peak hours, expressed as an integer between 0 and 23, inclusive.
2067 Set to -1 to disable off-peak.
2069 *Default*: `-1` (disabled)
2071 `hbase.regionserver.thread.compaction.throttle`::
2072 There are two different thread pools for compactions, one for large compactions
2073 and the other for small compactions. This helps to keep compaction of lean tables
2074 (such as `hbase:meta`) fast. If a compaction is larger than this threshold,
2075 it goes into the large compaction pool. In most cases, the default value is
2078 *Default*: `2 x hbase.hstore.compaction.max x hbase.hregion.memstore.flush.size`
2079 (which defaults to `128`)
2081 `hbase.hregion.majorcompaction`::
2082 Time between major compactions, expressed in milliseconds. Set to 0 to disable
2083 time-based automatic major compactions. User-requested and size-based major
2084 compactions will still run. This value is multiplied by
2085 `hbase.hregion.majorcompaction.jitter` to cause compaction to start at a
2086 somewhat-random time during a given window of time.
2088 *Default*: 7 days (`604800000` milliseconds)
2090 `hbase.hregion.majorcompaction.jitter`::
2091 A multiplier applied to hbase.hregion.majorcompaction to cause compaction to
2092 occur a given amount of time either side of `hbase.hregion.majorcompaction`.
2093 The smaller the number, the closer the compactions will happen to the
2094 `hbase.hregion.majorcompaction` interval. Expressed as a floating-point decimal.
2098 [[compaction.file.selection.old]]
2099 ===== Compaction File Selection
2104 This section has been preserved for historical reasons and refers to the way compaction worked prior to HBase 0.96.x.
2105 You can still use this behavior if you enable <<compaction.ratiobasedcompactionpolicy.algorithm>>. For information on the way that compactions work in HBase 0.96.x and later, see <<compaction>>.
2108 To understand the core algorithm for StoreFile selection, there is some ASCII-art in the Store source code that will serve as useful reference.
2110 It has been copied below:
2119 * --|-|- |-|- |-|---_-------_------- minCompactSize
2120 * | | | | | | | | _ | |
2121 * | | | | | | | | | | | |
2122 * | | | | | | | | | | | |
2126 * `hbase.hstore.compaction.ratio` Ratio used in compaction file selection algorithm (default 1.2f).
2127 * `hbase.hstore.compaction.min` (in HBase v 0.90 this is called `hbase.hstore.compactionThreshold`) (files) Minimum number of StoreFiles per Store to be selected for a compaction to occur (default 2).
2128 * `hbase.hstore.compaction.max` (files) Maximum number of StoreFiles to compact per minor compaction (default 10).
2129 * `hbase.hstore.compaction.min.size` (bytes) Any StoreFile smaller than this setting with automatically be a candidate for compaction.
2130 Defaults to `hbase.hregion.memstore.flush.size` (128 mb).
2131 * `hbase.hstore.compaction.max.size` (.92) (bytes) Any StoreFile larger than this setting with automatically be excluded from compaction (default Long.MAX_VALUE).
2133 The minor compaction StoreFile selection logic is size based, and selects a file for compaction when the `file <= sum(smaller_files) * hbase.hstore.compaction.ratio`.
2135 [[compaction.file.selection.example1]]
2136 ====== Minor Compaction File Selection - Example #1 (Basic Example)
2138 This example mirrors an example from the unit test `TestCompactSelection`.
2140 * `hbase.hstore.compaction.ratio` = 1.0f
2141 * `hbase.hstore.compaction.min` = 3 (files)
2142 * `hbase.hstore.compaction.max` = 5 (files)
2143 * `hbase.hstore.compaction.min.size` = 10 (bytes)
2144 * `hbase.hstore.compaction.max.size` = 1000 (bytes)
2146 The following StoreFiles exist: 100, 50, 23, 12, and 12 bytes apiece (oldest to newest). With the above parameters, the files that would be selected for minor compaction are 23, 12, and 12.
2150 * 100 -> No, because sum(50, 23, 12, 12) * 1.0 = 97.
2151 * 50 -> No, because sum(23, 12, 12) * 1.0 = 47.
2152 * 23 -> Yes, because sum(12, 12) * 1.0 = 24.
2153 * 12 -> Yes, because the previous file has been included, and because this does not exceed the max-file limit of 5
2154 * 12 -> Yes, because the previous file had been included, and because this does not exceed the max-file limit of 5.
2156 [[compaction.file.selection.example2]]
2157 ====== Minor Compaction File Selection - Example #2 (Not Enough Files ToCompact)
2159 This example mirrors an example from the unit test `TestCompactSelection`.
2161 * `hbase.hstore.compaction.ratio` = 1.0f
2162 * `hbase.hstore.compaction.min` = 3 (files)
2163 * `hbase.hstore.compaction.max` = 5 (files)
2164 * `hbase.hstore.compaction.min.size` = 10 (bytes)
2165 * `hbase.hstore.compaction.max.size` = 1000 (bytes)
2167 The following StoreFiles exist: 100, 25, 12, and 12 bytes apiece (oldest to newest). With the above parameters, no compaction will be started.
2171 * 100 -> No, because sum(25, 12, 12) * 1.0 = 47
2172 * 25 -> No, because sum(12, 12) * 1.0 = 24
2173 * 12 -> No. Candidate because sum(12) * 1.0 = 12, there are only 2 files to compact and that is less than the threshold of 3
2174 * 12 -> No. Candidate because the previous StoreFile was, but there are not enough files to compact
2176 [[compaction.file.selection.example3]]
2177 ====== Minor Compaction File Selection - Example #3 (Limiting Files To Compact)
2179 This example mirrors an example from the unit test `TestCompactSelection`.
2181 * `hbase.hstore.compaction.ratio` = 1.0f
2182 * `hbase.hstore.compaction.min` = 3 (files)
2183 * `hbase.hstore.compaction.max` = 5 (files)
2184 * `hbase.hstore.compaction.min.size` = 10 (bytes)
2185 * `hbase.hstore.compaction.max.size` = 1000 (bytes)
2187 The following StoreFiles exist: 7, 6, 5, 4, 3, 2, and 1 bytes apiece (oldest to newest). With the above parameters, the files that would be selected for minor compaction are 7, 6, 5, 4, 3.
2191 * 7 -> Yes, because sum(6, 5, 4, 3, 2, 1) * 1.0 = 21.
2192 Also, 7 is less than the min-size
2193 * 6 -> Yes, because sum(5, 4, 3, 2, 1) * 1.0 = 15.
2194 Also, 6 is less than the min-size.
2195 * 5 -> Yes, because sum(4, 3, 2, 1) * 1.0 = 10.
2196 Also, 5 is less than the min-size.
2197 * 4 -> Yes, because sum(3, 2, 1) * 1.0 = 6.
2198 Also, 4 is less than the min-size.
2199 * 3 -> Yes, because sum(2, 1) * 1.0 = 3.
2200 Also, 3 is less than the min-size.
2202 Candidate because previous file was selected and 2 is less than the min-size, but the max-number of files to compact has been reached.
2204 Candidate because previous file was selected and 1 is less than the min-size, but max-number of files to compact has been reached.
2206 [[compaction.config.impact]]
2207 .Impact of Key Configuration Options
2209 NOTE: This information is now included in the configuration parameter table in <<compaction.parameters>>.
2212 ===== Date Tiered Compaction
2214 Date tiered compaction is a date-aware store file compaction strategy that is beneficial for time-range scans for time-series data.
2216 [[ops.date.tiered.when]]
2217 ===== When To Use Date Tiered Compactions
2219 Consider using Date Tiered Compaction for reads for limited time ranges, especially scans of recent data
2223 * random gets without a limited time range
2224 * frequent deletes and updates
2225 * Frequent out of order data writes creating long tails, especially writes with future timestamps
2226 * frequent bulk loads with heavily overlapping time ranges
2228 .Performance Improvements
2229 Performance testing has shown that the performance of time-range scans improve greatly for limited time ranges, especially scans of recent data.
2231 [[ops.date.tiered.enable]]
2232 ====== Enabling Date Tiered Compaction
2234 You can enable Date Tiered compaction for a table or a column family, by setting its `hbase.hstore.engine.class` to `org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine`.
2236 You also need to set `hbase.hstore.blockingStoreFiles` to a high number, such as 60, if using all default settings, rather than the default value of 12). Use 1.5~2 x projected file count if changing the parameters, Projected file count = windows per tier x tier count + incoming window min + files older than max age
2238 You also need to set `hbase.hstore.compaction.max` to the same value as `hbase.hstore.blockingStoreFiles` to unblock major compaction.
2240 .Procedure: Enable Date Tiered Compaction
2241 . Run one of following commands in the HBase shell.
2242 Replace the table name `orders_table` with the name of your table.
2246 alter 'orders_table', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine', 'hbase.hstore.blockingStoreFiles' => '60', 'hbase.hstore.compaction.min'=>'2', 'hbase.hstore.compaction.max'=>'60'}
2247 alter 'orders_table', {NAME => 'blobs_cf', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine', 'hbase.hstore.blockingStoreFiles' => '60', 'hbase.hstore.compaction.min'=>'2', 'hbase.hstore.compaction.max'=>'60'}}
2248 create 'orders_table', 'blobs_cf', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine', 'hbase.hstore.blockingStoreFiles' => '60', 'hbase.hstore.compaction.min'=>'2', 'hbase.hstore.compaction.max'=>'60'}
2251 . Configure other options if needed.
2252 See <<ops.date.tiered.config>> for more information.
2254 .Procedure: Disable Date Tiered Compaction
2255 . Set the `hbase.hstore.engine.class` option to either nil or `org.apache.hadoop.hbase.regionserver.DefaultStoreEngine`.
2256 Either option has the same effect.
2257 Make sure you set the other options you changed to the original settings too.
2261 alter 'orders_table', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.DefaultStoreEngine', 'hbase.hstore.blockingStoreFiles' => '12', 'hbase.hstore.compaction.min'=>'6', 'hbase.hstore.compaction.max'=>'12'}}
2264 When you change the store engine either way, a major compaction will likely be performed on most regions.
2265 This is not necessary on new tables.
2267 [[ops.date.tiered.config]]
2268 ====== Configuring Date Tiered Compaction
2270 Each of the settings for date tiered compaction should be configured at the table or column family level.
2271 If you use HBase shell, the general command pattern is as follows:
2275 alter 'orders_table', CONFIGURATION => {'key' => 'value', ..., 'key' => 'value'}}
2278 [[ops.date.tiered.config.parameters]]
2281 You can configure your date tiers by changing the settings for the following parameters:
2283 .Date Tier Parameters
2284 [cols="1,1a", frame="all", options="header"]
2289 |`hbase.hstore.compaction.date.tiered.max.storefile.age.millis`
2290 |Files with max-timestamp smaller than this will no longer be compacted.Default at Long.MAX_VALUE.
2292 | `hbase.hstore.compaction.date.tiered.base.window.millis`
2293 | Base window size in milliseconds. Default at 6 hours.
2295 | `hbase.hstore.compaction.date.tiered.windows.per.tier`
2296 | Number of windows per tier. Default at 4.
2298 | `hbase.hstore.compaction.date.tiered.incoming.window.min`
2299 | Minimal number of files to compact in the incoming window. Set it to expected number of files in the window to avoid wasteful compaction. Default at 6.
2301 | `hbase.hstore.compaction.date.tiered.window.policy.class`
2302 | The policy to select store files within the same time window. It doesn’t apply to the incoming window. Default at exploring compaction. This is to avoid wasteful compaction.
2305 [[ops.date.tiered.config.compaction.throttler]]
2306 .Compaction Throttler
2308 With tiered compaction all servers in the cluster will promote windows to higher tier at the same time, so using a compaction throttle is recommended:
2309 Set `hbase.regionserver.throughput.controller` to `org.apache.hadoop.hbase.regionserver.compactions.PressureAwareCompactionThroughputController`.
2311 NOTE: For more information about date tiered compaction, please refer to the design specification at https://docs.google.com/document/d/1_AmlNb2N8Us1xICsTeGDLKIqL6T-oHoRLZ323MG_uy8
2313 ===== Experimental: Stripe Compactions
2315 Stripe compactions is an experimental feature added in HBase 0.98 which aims to improve compactions for large regions or non-uniformly distributed row keys.
2316 In order to achieve smaller and/or more granular compactions, the StoreFiles within a region are maintained separately for several row-key sub-ranges, or "stripes", of the region.
2317 The stripes are transparent to the rest of HBase, so other operations on the HFiles or data work without modification.
2319 Stripe compactions change the HFile layout, creating sub-regions within regions.
2320 These sub-regions are easier to compact, and should result in fewer major compactions.
2321 This approach alleviates some of the challenges of larger regions.
2323 Stripe compaction is fully compatible with <<compaction>> and works in conjunction with either the ExploringCompactionPolicy or RatioBasedCompactionPolicy.
2324 It can be enabled for existing tables, and the table will continue to operate normally if it is disabled later.
2327 ===== When To Use Stripe Compactions
2329 Consider using stripe compaction if you have either of the following:
2332 You can get the positive effects of smaller regions without additional overhead for MemStore and region management overhead.
2333 * Non-uniform keys, such as time dimension in a key.
2334 Only the stripes receiving the new keys will need to compact.
2335 Old data will not compact as often, if at all
2337 .Performance Improvements
2338 Performance testing has shown that the performance of reads improves somewhat, and variability of performance of reads and writes is greatly reduced.
2339 An overall long-term performance improvement is seen on large non-uniform-row key regions, such as a hash-prefixed timestamp key.
2340 These performance gains are the most dramatic on a table which is already large.
2341 It is possible that the performance improvement might extend to region splits.
2343 [[ops.stripe.enable]]
2344 ====== Enabling Stripe Compaction
2346 You can enable stripe compaction for a table or a column family, by setting its `hbase.hstore.engine.class` to `org.apache.hadoop.hbase.regionserver.StripeStoreEngine`.
2347 You also need to set the `hbase.hstore.blockingStoreFiles` to a high number, such as 100 (rather than the default value of 10).
2349 .Procedure: Enable Stripe Compaction
2350 . Run one of following commands in the HBase shell.
2351 Replace the table name `orders_table` with the name of your table.
2355 alter 'orders_table', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.StripeStoreEngine', 'hbase.hstore.blockingStoreFiles' => '100'}
2356 alter 'orders_table', {NAME => 'blobs_cf', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.StripeStoreEngine', 'hbase.hstore.blockingStoreFiles' => '100'}}
2357 create 'orders_table', 'blobs_cf', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.StripeStoreEngine', 'hbase.hstore.blockingStoreFiles' => '100'}
2360 . Configure other options if needed.
2361 See <<ops.stripe.config>> for more information.
2364 .Procedure: Disable Stripe Compaction
2365 . Set the `hbase.hstore.engine.class` option to either nil or `org.apache.hadoop.hbase.regionserver.DefaultStoreEngine`.
2366 Either option has the same effect.
2370 alter 'orders_table', CONFIGURATION => {'hbase.hstore.engine.class' => 'rg.apache.hadoop.hbase.regionserver.DefaultStoreEngine'}
2375 When you enable a large table after changing the store engine either way, a major compaction will likely be performed on most regions.
2376 This is not necessary on new tables.
2378 [[ops.stripe.config]]
2379 ====== Configuring Stripe Compaction
2381 Each of the settings for stripe compaction should be configured at the table or column family level.
2382 If you use HBase shell, the general command pattern is as follows:
2386 alter 'orders_table', CONFIGURATION => {'key' => 'value', ..., 'key' => 'value'}}
2389 [[ops.stripe.config.sizing]]
2390 .Region and stripe sizing
2392 You can configure your stripe sizing based upon your region sizing.
2393 By default, your new regions will start with one stripe.
2394 On the next compaction after the stripe has grown too large (16 x MemStore flushes size), it is split into two stripes.
2395 Stripe splitting continues as the region grows, until the region is large enough to split.
2397 You can improve this pattern for your own data.
2398 A good rule is to aim for a stripe size of at least 1 GB, and about 8-12 stripes for uniform row keys.
2399 For example, if your regions are 30 GB, 12 x 2.5 GB stripes might be a good starting point.
2401 .Stripe Sizing Settings
2402 [cols="1,1a", frame="all", options="header"]
2407 |`hbase.store.stripe.initialStripeCount`
2408 |The number of stripes to create when stripe compaction is enabled. You can use it as follows:
2410 * For relatively uniform row keys, if you know the approximate
2411 target number of stripes from the above, you can avoid some
2412 splitting overhead by starting with several stripes (2, 5, 10...).
2413 If the early data is not representative of overall row key
2414 distribution, this will not be as efficient.
2416 * For existing tables with a large amount of data, this setting
2417 will effectively pre-split your stripes.
2419 * For keys such as hash-prefixed sequential keys, with more than
2420 one hash prefix per region, pre-splitting may make sense.
2423 | `hbase.store.stripe.sizeToSplit`
2424 | The maximum size a stripe grows before splitting. Use this in
2425 conjunction with `hbase.store.stripe.splitPartCount` to
2426 control the target stripe size (`sizeToSplit = splitPartsCount * target
2427 stripe size`), according to the above sizing considerations.
2429 | `hbase.store.stripe.splitPartCount`
2430 | The number of new stripes to create when splitting a stripe. The default is 2, which is appropriate for most cases. For non-uniform row keys, you can experiment with increasing the number to 3 or 4, to isolate the arriving updates into narrower slice of the region without additional splits being required.
2433 [[ops.stripe.config.memstore]]
2434 .MemStore Size Settings
2436 By default, the flush creates several files from one MemStore, according to existing stripe boundaries and row keys to flush.
2437 This approach minimizes write amplification, but can be undesirable if the MemStore is small and there are many stripes, because the files will be too small.
2439 In this type of situation, you can set `hbase.store.stripe.compaction.flushToL0` to `true`.
2440 This will cause a MemStore flush to create a single file instead.
2441 When at least `hbase.store.stripe.compaction.minFilesL0` such files (by default, 4) accumulate, they will be compacted into striped files.
2443 [[ops.stripe.config.compact]]
2444 .Normal Compaction Configuration and Stripe Compaction
2446 All the settings that apply to normal compactions (see <<compaction.parameters>>) apply to stripe compactions.
2447 The exceptions are the minimum and maximum number of files, which are set to higher values by default because the files in stripes are smaller.
2448 To control these for stripe compactions, use `hbase.store.stripe.compaction.minFiles` and `hbase.store.stripe.compaction.maxFiles`, rather than `hbase.hstore.compaction.min` and `hbase.hstore.compaction.max`.
2453 [[arch.bulk.load.overview]]
2456 HBase includes several methods of loading data into tables.
2457 The most straightforward method is to either use the `TableOutputFormat` class from a MapReduce job, or use the normal client APIs; however, these are not always the most efficient methods.
2459 The bulk load feature uses a MapReduce job to output table data in HBase's internal data format, and then directly loads the generated StoreFiles into a running cluster.
2460 Using bulk load will use less CPU and network resources than simply using the HBase API.
2462 [[arch.bulk.load.arch]]
2463 === Bulk Load Architecture
2465 The HBase bulk load process consists of two main steps.
2467 [[arch.bulk.load.prep]]
2468 ==== Preparing data via a MapReduce job
2470 The first step of a bulk load is to generate HBase data files (StoreFiles) from a MapReduce job using `HFileOutputFormat2`.
2471 This output format writes out data in HBase's internal storage format so that they can be later loaded very efficiently into the cluster.
2473 In order to function efficiently, `HFileOutputFormat2` must be configured such that each output HFile fits within a single region.
2474 In order to do this, jobs whose output will be bulk loaded into HBase use Hadoop's `TotalOrderPartitioner` class to partition the map output into disjoint ranges of the key space, corresponding to the key ranges of the regions in the table.
2476 `HFileOutputFormat2` includes a convenience function, `configureIncrementalLoad()`, which automatically sets up a `TotalOrderPartitioner` based on the current region boundaries of a table.
2478 [[arch.bulk.load.complete]]
2479 ==== Completing the data load
2481 After a data import has been prepared, either by using the `importtsv` tool with the "`importtsv.bulk.output`" option or by some other MapReduce job using the `HFileOutputFormat`, the `completebulkload` tool is used to import the data into the running cluster.
2482 This command line tool iterates through the prepared data files, and for each one determines the region the file belongs to.
2483 It then contacts the appropriate RegionServer which adopts the HFile, moving it into its storage directory and making the data available to clients.
2485 If the region boundaries have changed during the course of bulk load preparation, or between the preparation and completion steps, the `completebulkload` utility will automatically split the data files into pieces corresponding to the new boundaries.
2486 This process is not optimally efficient, so users should take care to minimize the delay between preparing a bulk load and importing it into the cluster, especially if other clients are simultaneously loading data through other means.
2490 $ hadoop jar hbase-server-VERSION.jar completebulkload [-c /path/to/hbase/config/hbase-site.xml] /user/todd/myoutput mytable
2493 The `-c config-file` option can be used to specify a file containing the appropriate hbase parameters (e.g., hbase-site.xml) if not supplied already on the CLASSPATH (In addition, the CLASSPATH must contain the directory that has the zookeeper configuration file if zookeeper is NOT managed by HBase).
2495 NOTE: If the target table does not already exist in HBase, this tool will create the table automatically.
2498 [[arch.bulk.load.also]]
2501 For more information about the referenced utilities, see <<importtsv>> and <<completebulkload>>.
2503 See link:http://blog.cloudera.com/blog/2013/09/how-to-use-hbase-bulk-loading-and-why/[How-to: Use HBase Bulk Loading, and Why] for a recent blog on current state of bulk loading.
2505 [[arch.bulk.load.adv]]
2508 Although the `importtsv` tool is useful in many cases, advanced users may want to generate data programmatically, or import data from other formats.
2509 To get started doing so, dig into `ImportTsv.java` and check the JavaDoc for HFileOutputFormat.
2511 The import step of the bulk load can also be done programmatically.
2512 See the `LoadIncrementalHFiles` class for more information.
2514 [[arch.bulk.load.replication]]
2515 === Bulk Loading Replication
2516 HBASE-13153 adds replication support for bulk loaded HFiles, available since HBase 1.3/2.0. This feature is enabled by setting `hbase.replication.bulkload.enabled` to `true` (default is `false`).
2517 You also need to copy the source cluster configuration files to the destination cluster.
2519 Additional configurations are required too:
2521 . `hbase.replication.source.fs.conf.provider`
2523 This defines the class which loads the source cluster file system client configuration in the destination cluster. This should be configured for all the RS in the destination cluster. Default is `org.apache.hadoop.hbase.replication.regionserver.DefaultSourceFSConfigurationProvider`.
2525 . `hbase.replication.conf.dir`
2527 This represents the base directory where the file system client configurations of the source cluster are copied to the destination cluster. This should be configured for all the RS in the destination cluster. Default is `$HBASE_CONF_DIR`.
2529 . `hbase.replication.cluster.id`
2531 This configuration is required in the cluster where replication for bulk loaded data is enabled. A source cluster is uniquely identified by the destination cluster using this id. This should be configured for all the RS in the source cluster configuration file for all the RS.
2536 For example: If source cluster FS client configurations are copied to the destination cluster under directory `/home/user/dc1/`, then `hbase.replication.cluster.id` should be configured as `dc1` and `hbase.replication.conf.dir` as `/home/user`.
2538 NOTE: `DefaultSourceFSConfigurationProvider` supports only `xml` type files. It loads source cluster FS client configuration only once, so if source cluster FS client configuration files are updated, every peer(s) cluster RS must be restarted to reload the configuration.
2543 As HBase runs on HDFS (and each StoreFile is written as a file on HDFS), it is important to have an understanding of the HDFS Architecture especially in terms of how it stores files, handles failovers, and replicates blocks.
2545 See the Hadoop documentation on link:https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html[HDFS Architecture] for more information.
2550 The NameNode is responsible for maintaining the filesystem metadata.
2551 See the above HDFS Architecture link for more information.
2556 The DataNodes are responsible for storing HDFS blocks.
2557 See the above HDFS Architecture link for more information.
2559 [[arch.timelineconsistent.reads]]
2560 == Timeline-consistent High Available Reads
2562 NOTE: The current <<amv2, Assignment Manager V2>> does not work well with region replica, so this feature maybe broken. Use it with caution.
2564 [[casestudies.timelineconsistent.intro]]
2567 HBase, architecturally, always had the strong consistency guarantee from the start.
2568 All reads and writes are routed through a single region server, which guarantees that all writes happen in an order, and all reads are seeing the most recent committed data.
2571 However, because of this single homing of the reads to a single location, if the server becomes unavailable, the regions of the table that were hosted in the region server become unavailable for some time.
2572 There are three phases in the region recovery process - detection, assignment, and recovery.
2573 Of these, the detection is usually the longest and is presently in the order of 20-30 seconds depending on the ZooKeeper session timeout.
2574 During this time and before the recovery is complete, the clients will not be able to read the region data.
2576 However, for some use cases, either the data may be read-only, or doing reads against some stale data is acceptable.
2577 With timeline-consistent high available reads, HBase can be used for these kind of latency-sensitive use cases where the application can expect to have a time bound on the read completion.
2580 For achieving high availability for reads, HBase provides a feature called _region replication_. In this model, for each region of a table, there will be multiple replicas that are opened in different RegionServers.
2581 By default, the region replication is set to 1, so only a single region replica is deployed and there will not be any changes from the original model.
2582 If region replication is set to 2 or more, then the master will assign replicas of the regions of the table.
2583 The Load Balancer ensures that the region replicas are not co-hosted in the same region servers and also in the same rack (if possible).
2585 All of the replicas for a single region will have a unique replica_id, starting from 0.
2586 The region replica having replica_id==0 is called the primary region, and the others _secondary regions_ or secondaries.
2587 Only the primary can accept writes from the client, and the primary will always contain the latest changes.
2588 Since all writes still have to go through the primary region, the writes are not highly-available (meaning they might block for some time if the region becomes unavailable).
2591 === Timeline Consistency
2593 With this feature, HBase introduces a Consistency definition, which can be provided per read operation (get or scan).
2596 public enum Consistency {
2601 `Consistency.STRONG` is the default consistency model provided by HBase.
2602 In case the table has region replication = 1, or in a table with region replicas but the reads are done with this consistency, the read is always performed by the primary regions, so that there will not be any change from the previous behaviour, and the client always observes the latest data.
2605 In case a read is performed with `Consistency.TIMELINE`, then the read RPC will be sent to the primary region server first.
2606 After a short interval (`hbase.client.primaryCallTimeout.get`, 10ms by default), parallel RPC for secondary region replicas will also be sent if the primary does not respond back.
2607 After this, the result is returned from whichever RPC is finished first.
2608 If the response came back from the primary region replica, we can always know that the data is latest.
2609 For this Result.isStale() API has been added to inspect the staleness.
2610 If the result is from a secondary region, then Result.isStale() will be set to true.
2611 The user can then inspect this field to possibly reason about the data.
2614 In terms of semantics, TIMELINE consistency as implemented by HBase differs from pure eventual consistency in these respects:
2616 * Single homed and ordered updates: Region replication or not, on the write side, there is still only 1 defined replica (primary) which can accept writes.
2617 This replica is responsible for ordering the edits and preventing conflicts.
2618 This guarantees that two different writes are not committed at the same time by different replicas and the data diverges.
2619 With this, there is no need to do read-repair or last-timestamp-wins kind of conflict resolution.
2620 * The secondaries also apply the edits in the order that the primary committed them.
2621 This way the secondaries will contain a snapshot of the primaries data at any point in time.
2622 This is similar to RDBMS replications and even HBase's own multi-datacenter replication, however in a single cluster.
2623 * On the read side, the client can detect whether the read is coming from up-to-date data or is stale data.
2624 Also, the client can issue reads with different consistency requirements on a per-operation basis to ensure its own semantic guarantees.
2625 * The client can still observe edits out-of-order, and can go back in time, if it observes reads from one secondary replica first, then another secondary replica.
2626 There is no stickiness to region replicas or a transaction-id based guarantee.
2627 If required, this can be implemented later though.
2629 .Timeline Consistency
2630 image::timeline_consistency.png[Timeline Consistency]
2632 To better understand the TIMELINE semantics, let's look at the above diagram.
2633 Let's say that there are two clients, and the first one writes x=1 at first, then x=2 and x=3 later.
2634 As above, all writes are handled by the primary region replica.
2635 The writes are saved in the write ahead log (WAL), and replicated to the other replicas asynchronously.
2636 In the above diagram, notice that replica_id=1 received 2 updates, and its data shows that x=2, while the replica_id=2 only received a single update, and its data shows that x=1.
2639 If client1 reads with STRONG consistency, it will only talk with the replica_id=0, and thus is guaranteed to observe the latest value of x=3.
2640 In case of a client issuing TIMELINE consistency reads, the RPC will go to all replicas (after primary timeout) and the result from the first response will be returned back.
2641 Thus the client can see either 1, 2 or 3 as the value of x.
2642 Let's say that the primary region has failed and log replication cannot continue for some time.
2643 If the client does multiple reads with TIMELINE consistency, she can observe x=2 first, then x=1, and so on.
2648 Having secondary regions hosted for read availability comes with some tradeoffs which should be carefully evaluated per use case.
2649 Following are advantages and disadvantages.
2652 * High availability for read-only tables
2653 * High availability for stale reads
2654 * Ability to do very low latency reads with very high percentile (99.9%+) latencies for stale reads
2657 * Double / Triple MemStore usage (depending on region replication count) for tables with region replication > 1
2658 * Increased block cache usage
2659 * Extra network traffic for log replication
2660 * Extra backup RPCs for replicas
2662 To serve the region data from multiple replicas, HBase opens the regions in secondary mode in the region servers.
2663 The regions opened in secondary mode will share the same data files with the primary region replica, however each secondary region replica will have its own MemStore to keep the unflushed data (only primary region can do flushes). Also to serve reads from secondary regions, the blocks of data files may be also cached in the block caches for the secondary regions.
2665 === Where is the code
2666 This feature is delivered in two phases, Phase 1 and 2. The first phase is done in time for HBase-1.0.0 release. Meaning that using HBase-1.0.x, you can use all the features that are marked for Phase 1. Phase 2 is committed in HBase-1.1.0, meaning all HBase versions after 1.1.0 should contain Phase 2 items.
2668 === Propagating writes to region replicas
2669 As discussed above writes only go to the primary region replica. For propagating the writes from the primary region replica to the secondaries, there are two different mechanisms. For read-only tables, you do not need to use any of the following methods. Disabling and enabling the table should make the data available in all region replicas. For mutable tables, you have to use *only* one of the following mechanisms: storefile refresher, or async wal replication. The latter is recommended.
2671 ==== StoreFile Refresher
2672 The first mechanism is store file refresher which is introduced in HBase-1.0+. Store file refresher is a thread per region server, which runs periodically, and does a refresh operation for the store files of the primary region for the secondary region replicas. If enabled, the refresher will ensure that the secondary region replicas see the new flushed, compacted or bulk loaded files from the primary region in a timely manner. However, this means that only flushed data can be read back from the secondary region replicas, and after the refresher is run, making the secondaries lag behind the primary for an a longer time.
2674 For turning this feature on, you should configure `hbase.regionserver.storefile.refresh.period` to a non-zero value. See Configuration section below.
2676 ==== Asnyc WAL replication
2677 The second mechanism for propagation of writes to secondaries is done via “Async WAL Replication” feature and is only available in HBase-1.1+. This works similarly to HBase’s multi-datacenter replication, but instead the data from a region is replicated to the secondary regions. Each secondary replica always receives and observes the writes in the same order that the primary region committed them. In some sense, this design can be thought of as “in-cluster replication”, where instead of replicating to a different datacenter, the data goes to secondary regions to keep secondary region’s in-memory state up to date. The data files are shared between the primary region and the other replicas, so that there is no extra storage overhead. However, the secondary regions will have recent non-flushed data in their memstores, which increases the memory overhead. The primary region writes flush, compaction, and bulk load events to its WAL as well, which are also replicated through wal replication to secondaries. When they observe the flush/compaction or bulk load event, the secondary regions replay the event to pick up the new files and drop the old ones.
2679 Committing writes in the same order as in primary ensures that the secondaries won’t diverge from the primary regions data, but since the log replication is asynchronous, the data might still be stale in secondary regions. Since this feature works as a replication endpoint, the performance and latency characteristics is expected to be similar to inter-cluster replication.
2681 Async WAL Replication is *disabled* by default. You can enable this feature by setting `hbase.region.replica.replication.enabled` to `true`.
2682 Asyn WAL Replication feature will add a new replication peer named `region_replica_replication` as a replication peer when you create a table with region replication > 1 for the first time. Once enabled, if you want to disable this feature, you need to do two actions:
2683 * Set configuration property `hbase.region.replica.replication.enabled` to false in `hbase-site.xml` (see Configuration section below)
2684 * Disable the replication peer named `region_replica_replication` in the cluster using hbase shell or `Admin` class:
2687 hbase> disable_peer 'region_replica_replication'
2691 In both of the write propagation approaches mentioned above, store files of the primary will be opened in secondaries independent of the primary region. So for files that the primary compacted away, the secondaries might still be referring to these files for reading. Both features are using HFileLinks to refer to files, but there is no protection (yet) for guaranteeing that the file will not be deleted prematurely. Thus, as a guard, you should set the configuration property `hbase.master.hfilecleaner.ttl` to a larger value, such as 1 hour to guarantee that you will not receive IOExceptions for requests going to replicas.
2693 === Region replication for META table’s region
2694 Currently, Async WAL Replication is not done for the META table’s WAL. The meta table’s secondary replicas still refreshes themselves from the persistent store files. Hence the `hbase.regionserver.meta.storefile.refresh.period` needs to be set to a certain non-zero value for refreshing the meta store files. Note that this configuration is configured differently than
2695 `hbase.regionserver.storefile.refresh.period`.
2697 === Memory accounting
2698 The secondary region replicas refer to the data files of the primary region replica, but they have their own memstores (in HBase-1.1+) and uses block cache as well. However, one distinction is that the secondary region replicas cannot flush the data when there is memory pressure for their memstores. They can only free up memstore memory when the primary region does a flush and this flush is replicated to the secondary. Since in a region server hosting primary replicas for some regions and secondaries for some others, the secondaries might cause extra flushes to the primary regions in the same host. In extreme situations, there can be no memory left for adding new writes coming from the primary via wal replication. For unblocking this situation (and since secondary cannot flush by itself), the secondary is allowed to do a “store file refresh” by doing a file system list operation to pick up new files from primary, and possibly dropping its memstore. This refresh will only be performed if the memstore size of the biggest secondary region replica is at least `hbase.region.replica.storefile.refresh.memstore.multiplier` (default 4) times bigger than the biggest memstore of a primary replica. One caveat is that if this is performed, the secondary can observe partial row updates across column families (since column families are flushed independently). The default should be good to not do this operation frequently. You can set this value to a large number to disable this feature if desired, but be warned that it might cause the replication to block forever.
2700 === Secondary replica failover
2701 When a secondary region replica first comes online, or fails over, it may have served some edits from its memstore. Since the recovery is handled differently for secondary replicas, the secondary has to ensure that it does not go back in time before it starts serving requests after assignment. For doing that, the secondary waits until it observes a full flush cycle (start flush, commit flush) or a “region open event” replicated from the primary. Until this happens, the secondary region replica will reject all read requests by throwing an IOException with message “The region's reads are disabled”. However, the other replicas will probably still be available to read, thus not causing any impact for the rpc with TIMELINE consistency. To facilitate faster recovery, the secondary region will trigger a flush request from the primary when it is opened. The configuration property `hbase.region.replica.wait.for.primary.flush` (enabled by default) can be used to disable this feature if needed.
2706 === Configuration properties
2708 To use highly available reads, you should set the following properties in `hbase-site.xml` file.
2709 There is no specific configuration to enable or disable region replicas.
2710 Instead you can change the number of region replicas per table to increase or decrease at the table creation or with alter table. The following configuration is for using async wal replication and using meta replicas of 3.
2713 ==== Server side properties
2718 <name>hbase.regionserver.storefile.refresh.period</name>
2721 The period (in milliseconds) for refreshing the store files for the secondary regions. 0 means this feature is disabled. Secondary regions sees new files (from flushes and compactions) from primary once the secondary region refreshes the list of files in the region (there is no notification mechanism). But too frequent refreshes might cause extra Namenode pressure. If the files cannot be refreshed for longer than HFile TTL (hbase.master.hfilecleaner.ttl) the requests are rejected. Configuring HFile TTL to a larger value is also recommended with this setting.
2726 <name>hbase.regionserver.meta.storefile.refresh.period</name>
2727 <value>300000</value>
2729 The period (in milliseconds) for refreshing the store files for the hbase:meta tables secondary regions. 0 means this feature is disabled. Secondary regions sees new files (from flushes and compactions) from primary once the secondary region refreshes the list of files in the region (there is no notification mechanism). But too frequent refreshes might cause extra Namenode pressure. If the files cannot be refreshed for longer than HFile TTL (hbase.master.hfilecleaner.ttl) the requests are rejected. Configuring HFile TTL to a larger value is also recommended with this setting. This should be a non-zero number if meta replicas are enabled (via hbase.meta.replica.count set to greater than 1).
2734 <name>hbase.region.replica.replication.enabled</name>
2737 Whether asynchronous WAL replication to the secondary region replicas is enabled or not. If this is enabled, a replication peer named "region_replica_replication" will be created which will tail the logs and replicate the mutations to region replicas for tables that have region replication > 1. If this is enabled once, disabling this replication also requires disabling the replication peer using shell or Admin java class. Replication to secondary region replicas works over standard inter-cluster replication.
2741 <name>hbase.region.replica.replication.memstore.enabled</name>
2744 If you set this to `false`, replicas do not receive memstore updates from
2745 the primary RegionServer. If you set this to `true`, you can still disable
2746 memstore replication on a per-table basis, by setting the table's
2747 `REGION_MEMSTORE_REPLICATION` configuration property to `false`. If
2748 memstore replication is disabled, the secondaries will only receive
2749 updates for events like flushes and bulkloads, and will not have access to
2750 data which the primary has not yet flushed. This preserves the guarantee
2751 of row-level consistency, even when the read requests `Consistency.TIMELINE`.
2756 <name>hbase.master.hfilecleaner.ttl</name>
2757 <value>3600000</value>
2759 The period (in milliseconds) to keep store files in the archive folder before deleting them from the file system.</description>
2763 <name>hbase.meta.replica.count</name>
2766 Region replication count for the meta regions. Defaults to 1.
2772 <name>hbase.region.replica.storefile.refresh.memstore.multiplier</name>
2775 The multiplier for a “store file refresh” operation for the secondary region replica. If a region server has memory pressure, the secondary region will refresh it’s store files if the memstore size of the biggest secondary replica is bigger this many times than the memstore size of the biggest primary replica. Set this to a very big value to disable this feature (not recommended).
2780 <name>hbase.region.replica.wait.for.primary.flush</name>
2783 Whether to wait for observing a full flush cycle from the primary before start serving data in a secondary. Disabling this might cause the secondary region replicas to go back in time for reads between region movements.
2788 One thing to keep in mind also is that, region replica placement policy is only enforced by the `StochasticLoadBalancer` which is the default balancer.
2789 If you are using a custom load balancer property in hbase-site.xml (`hbase.master.loadbalancer.class`) replicas of regions might end up being hosted in the same server.
2791 ==== Client side properties
2793 Ensure to set the following for all clients (and servers) that will use region replicas.
2798 <name>hbase.ipc.client.specificThreadForWriting</name>
2801 Whether to enable interruption of RPC threads at the client side. This is required for region replicas with fallback RPC’s to secondary regions.
2805 <name>hbase.client.primaryCallTimeout.get</name>
2806 <value>10000</value>
2808 The timeout (in microseconds), before secondary fallback RPC’s are submitted for get requests with Consistency.TIMELINE to the secondary replicas of the regions. Defaults to 10ms. Setting this lower will increase the number of RPC’s, but will lower the p99 latencies.
2812 <name>hbase.client.primaryCallTimeout.multiget</name>
2813 <value>10000</value>
2815 The timeout (in microseconds), before secondary fallback RPC’s are submitted for multi-get requests (Table.get(List<Get>)) with Consistency.TIMELINE to the secondary replicas of the regions. Defaults to 10ms. Setting this lower will increase the number of RPC’s, but will lower the p99 latencies.
2819 <name>hbase.client.replicaCallTimeout.scan</name>
2820 <value>1000000</value>
2822 The timeout (in microseconds), before secondary fallback RPC’s are submitted for scan requests with Consistency.TIMELINE to the secondary replicas of the regions. Defaults to 1 sec. Setting this lower will increase the number of RPC’s, but will lower the p99 latencies.
2826 <name>hbase.meta.replicas.use</name>
2829 Whether to use meta table replicas or not. Default is false.
2834 Note HBase-1.0.x users should use `hbase.ipc.client.allowsInterrupt` rather than `hbase.ipc.client.specificThreadForWriting`.
2838 In the masters user interface, the region replicas of a table are also shown together with the primary regions.
2839 You can notice that the replicas of a region will share the same start and end keys and the same region name prefix.
2840 The only difference would be the appended replica_id (which is encoded as hex), and the region encoded name will be different.
2841 You can also see the replica ids shown explicitly in the UI.
2843 === Creating a table with region replication
2845 Region replication is a per-table property.
2846 All tables have `REGION_REPLICATION = 1` by default, which means that there is only one replica per region.
2847 You can set and change the number of replicas per region of a table by supplying the `REGION_REPLICATION` property in the table descriptor.
2854 create 't1', 'f1', {REGION_REPLICATION => 2}
2858 put 't1', "r#{i}", 'f1:c1', i
2867 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(“test_table”));
2868 htd.setRegionReplication(2);
2870 admin.createTable(htd);
2873 You can also use `setRegionReplication()` and alter table to increase, decrease the region replication for a table.
2876 === Read API and Usage
2880 You can do reads in shell using a the Consistency.TIMELINE semantics as follows
2884 hbase(main):001:0> get 't1','r6', {CONSISTENCY => "TIMELINE"}
2887 You can simulate a region server pausing or becoming unavailable and do a read from the secondary replica:
2891 $ kill -STOP <pid or primary region server>
2893 hbase(main):001:0> get 't1','r6', {CONSISTENCY => "TIMELINE"}
2896 Using scans is also similar
2900 hbase> scan 't1', {CONSISTENCY => 'TIMELINE'}
2905 You can set the consistency for Gets and Scans and do requests as follows.
2909 Get get = new Get(row);
2910 get.setConsistency(Consistency.TIMELINE);
2912 Result result = table.get(get);
2915 You can also pass multiple gets:
2919 Get get1 = new Get(row);
2920 get1.setConsistency(Consistency.TIMELINE);
2922 ArrayList<Get> gets = new ArrayList<Get>();
2925 Result[] results = table.get(gets);
2932 Scan scan = new Scan();
2933 scan.setConsistency(Consistency.TIMELINE);
2935 ResultScanner scanner = table.getScanner(scan);
2938 You can inspect whether the results are coming from primary region or not by calling the `Result.isStale()` method:
2942 Result result = table.get(get);
2943 if (result.isStale()) {
2950 . More information about the design and implementation can be found at the jira issue: link:https://issues.apache.org/jira/browse/HBASE-10070[HBASE-10070]
2951 . HBaseCon 2014 talk: link:https://hbase.apache.org/www.hbasecon.com/#2014-PresentationsRecordings[HBase Read High Availability Using Timeline-Consistent Region Replicas] also contains some details and link:http://www.slideshare.net/enissoz/hbase-high-availability-for-reads-with-time[slides].
2953 ifdef::backend-docbook[]
2956 // Generated automatically by the DocBook toolchain.
2957 endif::backend-docbook[]